diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index c3d3193c2ac017..643f6eb23cd1b0 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1088,6 +1088,12 @@ DEFINE_Int32(grace_shutdown_wait_seconds, "120"); DEFINE_Int16(bitmap_serialize_version, "1"); +// group commit insert config +DEFINE_String(group_commit_replay_wal_dir, "./wal"); +DEFINE_Int32(group_commit_replay_wal_retry_num, "10"); +DEFINE_Int32(group_commit_replay_wal_retry_interval_seconds, "5"); +DEFINE_Int32(group_commit_sync_wal_batch, "10"); + // the count of thread to group commit insert DEFINE_Int32(group_commit_insert_threads, "10"); diff --git a/be/src/common/config.h b/be/src/common/config.h index fba48bf3f4e84f..349a8a93a91192 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1159,6 +1159,12 @@ DECLARE_Int32(grace_shutdown_wait_seconds); // BitmapValue serialize version. DECLARE_Int16(bitmap_serialize_version); +// group commit insert config +DECLARE_String(group_commit_replay_wal_dir); +DECLARE_Int32(group_commit_replay_wal_retry_num); +DECLARE_Int32(group_commit_replay_wal_retry_interval_seconds); +DECLARE_Int32(group_commit_sync_wal_batch); + // This config can be set to limit thread number in group commit insert thread pool. DECLARE_mInt32(group_commit_insert_threads); diff --git a/be/src/http/action/http_stream.cpp b/be/src/http/action/http_stream.cpp index f8068389b2a76e..62153c8f8ffd78 100644 --- a/be/src/http/action/http_stream.cpp +++ b/be/src/http/action/http_stream.cpp @@ -49,6 +49,7 @@ #include "runtime/client_cache.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" +#include "runtime/group_commit_mgr.h" #include "runtime/load_path_mgr.h" #include "runtime/plan_fragment_executor.h" #include "runtime/stream_load/new_load_stream_mgr.h" @@ -139,6 +140,11 @@ Status HttpStreamAction::_handle(HttpRequest* http_req, std::shared_ptrfuture.get()); + if (ctx->group_commit) { + LOG(INFO) << "skip commit because this is group commit, pipe_id=" << ctx->id.to_string(); + return Status::OK(); + } + int64_t commit_and_publish_start_time = MonotonicNanos(); RETURN_IF_ERROR(_exec_env->stream_load_executor()->commit_txn(ctx.get())); ctx->commit_and_publish_txn_cost_nanos = MonotonicNanos() - commit_and_publish_start_time; @@ -158,6 +164,7 @@ int HttpStreamAction::on_header(HttpRequest* req) { if (ctx->label.empty()) { ctx->label = generate_uuid_string(); } + ctx->group_commit = iequal(req->header(HTTP_GROUP_COMMIT), "true"); LOG(INFO) << "new income streaming load request." << ctx->brief() << " sql : " << req->header(HTTP_SQL); @@ -223,12 +230,13 @@ void HttpStreamAction::on_chunk_data(HttpRequest* req) { if (ctx == nullptr || !ctx->status.ok()) { return; } - + if (!req->header(HTTP_WAL_ID_KY).empty()) { + ctx->wal_id = std::stoll(req->header(HTTP_WAL_ID_KY)); + } struct evhttp_request* ev_req = req->get_evhttp_request(); auto evbuf = evhttp_request_get_input_buffer(ev_req); int64_t start_read_data_time = MonotonicNanos(); - while (evbuffer_get_length(evbuf) > 0) { auto bb = ByteBuffer::allocate(128 * 1024); auto remove_bytes = evbuffer_remove(evbuf, bb->ptr, bb->capacity); @@ -283,6 +291,7 @@ Status HttpStreamAction::_process_put(HttpRequest* http_req, request.__set_load_sql(http_req->header(HTTP_SQL)); request.__set_loadId(ctx->id.to_thrift()); request.__set_label(ctx->label); + request.__set_group_commit(ctx->group_commit); if (_exec_env->master_info()->__isset.backend_id) { request.__set_backend_id(_exec_env->master_info()->backend_id); } else { @@ -306,6 +315,14 @@ Status HttpStreamAction::_process_put(HttpRequest* http_req, ctx->db = ctx->put_result.params.db_name; ctx->table = ctx->put_result.params.table_name; ctx->txn_id = ctx->put_result.params.txn_conf.txn_id; + ctx->put_result.params.__set_wal_id(ctx->wal_id); + + if (ctx->group_commit) { + ctx->db_id = ctx->put_result.db_id; + ctx->table_id = ctx->put_result.table_id; + ctx->schema_version = ctx->put_result.base_schema_version; + return _exec_env->group_commit_mgr()->group_commit_stream_load(ctx); + } return _exec_env->stream_load_executor()->execute_plan_fragment(ctx); } diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp index b4d53b74c437f0..be4d8703236b88 100644 --- a/be/src/http/action/stream_load.cpp +++ b/be/src/http/action/stream_load.cpp @@ -52,6 +52,7 @@ #include "olap/storage_engine.h" #include "runtime/client_cache.h" #include "runtime/exec_env.h" +#include "runtime/group_commit_mgr.h" #include "runtime/load_path_mgr.h" #include "runtime/message_body_sink.h" #include "runtime/stream_load/new_load_stream_mgr.h" @@ -153,6 +154,11 @@ Status StreamLoadAction::_handle(std::shared_ptr ctx) { // wait stream load finish RETURN_IF_ERROR(ctx->future.get()); + if (ctx->group_commit) { + LOG(INFO) << "skip commit because this is group commit, pipe_id=" << ctx->id.to_string(); + return Status::OK(); + } + if (ctx->two_phase_commit) { int64_t pre_commit_start_time = MonotonicNanos(); RETURN_IF_ERROR(_exec_env->stream_load_executor()->pre_commit_txn(ctx.get())); @@ -178,8 +184,16 @@ int StreamLoadAction::on_header(HttpRequest* req) { url_decode(req->param(HTTP_DB_KEY), &ctx->db); url_decode(req->param(HTTP_TABLE_KEY), &ctx->table); ctx->label = req->header(HTTP_LABEL_KEY); - if (ctx->label.empty()) { - ctx->label = generate_uuid_string(); + Status st = Status::OK(); + if (iequal(req->header(HTTP_GROUP_COMMIT), "true")) { + if (!ctx->label.empty()) { + st = Status::InternalError("label and group_commit can't be set at the same time"); + } + ctx->group_commit = true; + } else { + if (ctx->label.empty()) { + ctx->label = generate_uuid_string(); + } } ctx->two_phase_commit = req->header(HTTP_TWO_PHASE_COMMIT) == "true" ? true : false; @@ -187,7 +201,9 @@ int StreamLoadAction::on_header(HttpRequest* req) { LOG(INFO) << "new income streaming load request." << ctx->brief() << ", db=" << ctx->db << ", tbl=" << ctx->table; - auto st = _on_header(req, ctx); + if (st.ok()) { + st = _on_header(req, ctx); + } if (!st.ok()) { ctx->status = std::move(st); if (ctx->need_rollback) { @@ -287,9 +303,11 @@ Status StreamLoadAction::_on_header(HttpRequest* http_req, std::shared_ptrload_comment = http_req->header(HTTP_COMMENT); } // begin transaction - int64_t begin_txn_start_time = MonotonicNanos(); - RETURN_IF_ERROR(_exec_env->stream_load_executor()->begin_txn(ctx.get())); - ctx->begin_txn_cost_nanos = MonotonicNanos() - begin_txn_start_time; + if (!ctx->group_commit) { + int64_t begin_txn_start_time = MonotonicNanos(); + RETURN_IF_ERROR(_exec_env->stream_load_executor()->begin_txn(ctx.get())); + ctx->begin_txn_cost_nanos = MonotonicNanos() - begin_txn_start_time; + } // process put file return _process_put(http_req, ctx); @@ -555,6 +573,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, bool value = iequal(http_req->header(HTTP_MEMTABLE_ON_SINKNODE), "true"); request.__set_memtable_on_sink_node(value); } + request.__set_group_commit(ctx->group_commit); #ifndef BE_TEST // plan this load @@ -582,6 +601,13 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, return Status::OK(); } + if (ctx->group_commit) { + ctx->db_id = ctx->put_result.db_id; + ctx->table_id = ctx->put_result.table_id; + ctx->schema_version = ctx->put_result.base_schema_version; + return _exec_env->group_commit_mgr()->group_commit_stream_load(ctx); + } + return _exec_env->stream_load_executor()->execute_plan_fragment(ctx); } diff --git a/be/src/http/http_common.h b/be/src/http/http_common.h index bcbfa33e1011db..5a1550f48fcefc 100644 --- a/be/src/http/http_common.h +++ b/be/src/http/http_common.h @@ -63,5 +63,8 @@ static const std::string HTTP_TWO_PHASE_COMMIT = "two_phase_commit"; static const std::string HTTP_TXN_ID_KEY = "txn_id"; static const std::string HTTP_TXN_OPERATION_KEY = "txn_operation"; static const std::string HTTP_MEMTABLE_ON_SINKNODE = "memtable_on_sink_node"; +static const std::string HTTP_WAL_ID_KY = "wal_id"; +static const std::string HTTP_AUTH_CODE = "auth_code"; +static const std::string HTTP_GROUP_COMMIT = "group_commit"; } // namespace doris diff --git a/be/src/http/utils.cpp b/be/src/http/utils.cpp index fe11738d5ab8d2..f55b5d47696c11 100644 --- a/be/src/http/utils.cpp +++ b/be/src/http/utils.cpp @@ -30,6 +30,7 @@ #include "common/status.h" #include "common/utils.h" #include "http/http_channel.h" +#include "http/http_common.h" #include "http/http_headers.h" #include "http/http_method.h" #include "http/http_request.h" @@ -72,7 +73,12 @@ bool parse_basic_auth(const HttpRequest& req, std::string* user, std::string* pa bool parse_basic_auth(const HttpRequest& req, AuthInfo* auth) { auto& token = req.header("token"); - if (token.empty()) { + auto& auth_code = req.header(HTTP_AUTH_CODE); + if (!token.empty()) { + auth->token = token; + } else if (!auth_code.empty()) { + auth->auth_code = std::stoll(auth_code); + } else { std::string full_user; if (!parse_basic_auth(req, &full_user, &auth->passwd)) { return false; @@ -84,8 +90,6 @@ bool parse_basic_auth(const HttpRequest& req, AuthInfo* auth) { } else { auth->user = full_user; } - } else { - auth->token = token; } // set user ip diff --git a/be/src/olap/delete_bitmap_calculator.cpp b/be/src/olap/delete_bitmap_calculator.cpp index e529336747f82d..86d82783e4effa 100644 --- a/be/src/olap/delete_bitmap_calculator.cpp +++ b/be/src/olap/delete_bitmap_calculator.cpp @@ -120,6 +120,8 @@ bool MergeIndexDeleteBitmapCalculatorContext::Comparator::operator()( bool MergeIndexDeleteBitmapCalculatorContext::Comparator::is_key_same(Slice const& lhs, Slice const& rhs) const { + DCHECK(lhs.get_size() >= _sequence_length); + DCHECK(rhs.get_size() >= _sequence_length); auto lhs_without_seq = Slice(lhs.get_data(), lhs.get_size() - _sequence_length); auto rhs_without_seq = Slice(rhs.get_data(), rhs.get_size() - _sequence_length); return lhs_without_seq.compare(rhs_without_seq) == 0; @@ -154,7 +156,7 @@ Status MergeIndexDeleteBitmapCalculator::calculate_one(RowLocation& loc) { _heap->pop(); Slice cur_key; RETURN_IF_ERROR(cur_ctx->get_current_key(cur_key)); - if (_comparator.is_key_same(cur_key, _last_key)) { + if (!_last_key.empty() && _comparator.is_key_same(cur_key, _last_key)) { loc.segment_id = cur_ctx->segment_id(); loc.row_id = cur_ctx->row_id(); auto st = cur_ctx->advance(); diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp index 9cc3c94c09e726..63f96443064679 100644 --- a/be/src/olap/push_handler.cpp +++ b/be/src/olap/push_handler.cpp @@ -17,6 +17,7 @@ #include "olap/push_handler.h" +#include #include #include #include @@ -25,6 +26,7 @@ #include #include #include +#include #include #include @@ -138,6 +140,33 @@ Status PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TPushR DeletePredicatePB del_pred; TabletSchema tablet_schema; tablet_schema.copy_from(*tablet->tablet_schema()); + for (const auto& delete_cond : request.delete_conditions) { + if (!delete_cond.__isset.column_unique_id) { + LOG(WARNING) << "column=" << delete_cond.column_name + << " in predicate does not have uid, table id=" + << tablet_schema.table_id(); + // TODO(tsy): make it fail here after FE forbidding hard-link-schema-change + continue; + } + if (tablet_schema.field_index(delete_cond.column_unique_id) == -1) { + const auto& err_msg = + fmt::format("column id={} does not exists, table id={}", + delete_cond.column_unique_id, tablet_schema.table_id()); + LOG(WARNING) << err_msg; + DCHECK(false); + return Status::Aborted(err_msg); + } + if (tablet_schema.column_by_uid(delete_cond.column_unique_id).name() != + delete_cond.column_name) { + const auto& err_msg = fmt::format( + "colum name={} does not belongs to column uid={}, which column name={}", + delete_cond.column_name, delete_cond.column_unique_id, + tablet_schema.column_by_uid(delete_cond.column_unique_id).name()); + LOG(WARNING) << err_msg; + DCHECK(false); + return Status::Aborted(err_msg); + } + } if (!request.columns_desc.empty() && request.columns_desc[0].col_unique_id >= 0) { tablet_schema.clear_columns(); for (const auto& column_desc : request.columns_desc) { diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp index 1bfac900d2b592..988b39f02c756a 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -585,7 +585,17 @@ Status SegmentWriter::fill_missing_columns(vectorized::MutableColumns& mutable_f // build default value columns auto default_value_block = old_value_block.clone_empty(); auto mutable_default_value_columns = default_value_block.mutate_columns(); - if (has_default_or_nullable) { + + const vectorized::Int8* delete_sign_column_data = nullptr; + if (const vectorized::ColumnWithTypeAndName* delete_sign_column = + old_value_block.try_get_by_name(DELETE_SIGN); + delete_sign_column != nullptr && _tablet_schema->has_sequence_col()) { + auto& delete_sign_col = + reinterpret_cast(*(delete_sign_column->column)); + delete_sign_column_data = delete_sign_col.get_data().data(); + } + + if (has_default_or_nullable || delete_sign_column_data != nullptr) { for (auto i = 0; i < cids_missing.size(); ++i) { const auto& column = _tablet_schema->column(cids_missing[i]); if (column.has_default_value()) { @@ -600,7 +610,15 @@ Status SegmentWriter::fill_missing_columns(vectorized::MutableColumns& mutable_f // fill all missing value from mutable_old_columns, need to consider default value and null value for (auto idx = 0; idx < use_default_or_null_flag.size(); idx++) { - if (use_default_or_null_flag[idx]) { + // `use_default_or_null_flag[idx] == true` doesn't mean that we should read values from the old row + // for the missing columns. For example, if a table has sequence column, the rows with DELETE_SIGN column + // marked will not be marked in delete bitmap(see https://github.com/apache/doris/pull/24011), so it will + // be found in Tablet::lookup_row_key() and `use_default_or_null_flag[idx]` will be false. But we should not + // read values from old rows for missing values in this occasion. So we should read the DELETE_SIGN column + // to check if a row REALLY exists in the table. + if (use_default_or_null_flag[idx] || + (delete_sign_column_data != nullptr && + delete_sign_column_data[read_index[idx + segment_start_pos]] != 0)) { for (auto i = 0; i < cids_missing.size(); ++i) { // if the column has default value, fiil it with default value // otherwise, if the column is nullable, fill it with null value diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index ec90f4748feebc..4776ccc5c8f64a 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -119,6 +119,7 @@ #include "util/time.h" #include "util/trace.h" #include "util/uid_util.h" +#include "util/work_thread_pool.hpp" #include "vec/columns/column.h" #include "vec/columns/column_string.h" #include "vec/common/string_ref.h" @@ -174,7 +175,11 @@ struct WriteCooldownMetaExecutors { }; // Each executor is a mpsc to ensure uploads of the same tablet meta are not concurrent // FIXME(AlexYue): Use mpsc instead of `ThreadPool` with 1 thread - std::vector> _executors; + // We use PriorityThreadPool since it would call status inside it's `shutdown` function. + // Consider one situation where the StackTraceCache's singleton is detructed before + // this WriteCooldownMetaExecutors's singleton, then invoking the status would also call + // StackTraceCache which would then result in heap use after free like #23834 + std::vector> _executors; std::unordered_set _pending_tablets; std::mutex _latch; size_t _executor_nums; @@ -183,7 +188,7 @@ struct WriteCooldownMetaExecutors { WriteCooldownMetaExecutors::WriteCooldownMetaExecutors(size_t executor_nums) : _executor_nums(executor_nums) { for (size_t i = 0; i < _executor_nums; i++) { - std::unique_ptr pool; + std::unique_ptr pool; ThreadPoolBuilder("WriteCooldownMetaExecutor") .set_min_threads(1) .set_max_threads(1) @@ -230,7 +235,7 @@ void WriteCooldownMetaExecutors::WriteCooldownMetaExecutors::submit(TabletShared VLOG_DEBUG << "tablet " << t->tablet_id() << " is not cooldown replica"; }; - _executors[_get_executor_pos(tablet_id)]->submit_func( + _executors[_get_executor_pos(tablet_id)]->offer( [task = std::move(async_write_task)]() { task(); }); } diff --git a/be/src/olap/wal_manager.cpp b/be/src/olap/wal_manager.cpp new file mode 100644 index 00000000000000..8379c9e0d855a4 --- /dev/null +++ b/be/src/olap/wal_manager.cpp @@ -0,0 +1,252 @@ +// 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 "olap/wal_manager.h" + +#include + +#include +#include + +#include "io/fs/local_file_system.h" +#include "runtime/client_cache.h" +#include "runtime/fragment_mgr.h" +#include "runtime/plan_fragment_executor.h" +#include "runtime/stream_load/stream_load_context.h" +#include "util/path_util.h" +#include "util/thrift_rpc_helper.h" +#include "vec/exec/format/wal/wal_reader.h" + +namespace doris { +WalManager::WalManager(ExecEnv* exec_env, const std::string& wal_dir_list) + : _exec_env(exec_env), _stop_background_threads_latch(1) { + doris::vectorized::WalReader::string_split(wal_dir_list, ",", _wal_dirs); +} + +WalManager::~WalManager() { + _stop_background_threads_latch.count_down(); + if (_replay_thread) { + _replay_thread->join(); + } + LOG(INFO) << "WalManager is destoried"; +} +void WalManager::stop() { + _stop = true; + LOG(INFO) << "WalManager is stopped"; +} + +Status WalManager::init() { + bool exists = false; + for (auto wal_dir : _wal_dirs) { + std::string tmp_dir = wal_dir + "/tmp"; + LOG(INFO) << "wal_dir:" << wal_dir << ",tmp_dir:" << tmp_dir; + RETURN_IF_ERROR(io::global_local_filesystem()->exists(wal_dir, &exists)); + if (!exists) { + RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(wal_dir)); + } + RETURN_IF_ERROR(io::global_local_filesystem()->exists(tmp_dir, &exists)); + if (!exists) { + RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(tmp_dir)); + } + RETURN_IF_ERROR(scan_wals(wal_dir)); + } + return Thread::create( + "WalMgr", "replay_wal", [this]() { this->replay(); }, &_replay_thread); +} + +Status WalManager::add_wal_path(int64_t db_id, int64_t table_id, int64_t wal_id, + const std::string& label) { + std::string base_path = + _wal_dirs.size() == 1 ? _wal_dirs[0] : _wal_dirs[rand() % _wal_dirs.size()]; + std::stringstream ss; + ss << base_path << "/" << std::to_string(db_id) << "/" << std::to_string(table_id) << "/" + << std::to_string(wal_id) << "_" << label; + { + std::lock_guard wrlock(_wal_lock); + _wal_path_map.emplace(wal_id, ss.str()); + } + return Status::OK(); +} + +Status WalManager::get_wal_path(int64_t wal_id, std::string& wal_path) { + std::shared_lock rdlock(_wal_lock); + auto it = _wal_path_map.find(wal_id); + if (it != _wal_path_map.end()) { + wal_path = _wal_path_map[wal_id]; + } else { + return Status::InternalError("can not find wal_id {} in wal_path_map", wal_id); + } + return Status::OK(); +} + +Status WalManager::create_wal_reader(const std::string& wal_path, + std::shared_ptr& wal_reader) { + wal_reader = std::make_shared(wal_path); + RETURN_IF_ERROR(wal_reader->init()); + return Status::OK(); +} + +Status WalManager::create_wal_writer(int64_t wal_id, std::shared_ptr& wal_writer) { + std::string wal_path; + RETURN_IF_ERROR(get_wal_path(wal_id, wal_path)); + std::vector path_element; + doris::vectorized::WalReader::string_split(wal_path, "/", path_element); + std::stringstream ss; + for (int i = 0; i < path_element.size() - 1; i++) { + ss << path_element[i] << "/"; + } + std::string base_path = ss.str(); + bool exists = false; + RETURN_IF_ERROR(io::global_local_filesystem()->exists(base_path, &exists)); + if (!exists) { + RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(base_path)); + } + LOG(INFO) << "create wal " << wal_path; + wal_writer = std::make_shared(wal_path); + RETURN_IF_ERROR(wal_writer->init()); + return Status::OK(); +} + +Status WalManager::scan_wals(const std::string& wal_path) { + size_t count = 0; + bool exists = true; + std::vector dbs; + Status st = io::global_local_filesystem()->list(wal_path, false, &dbs, &exists); + if (!st.ok()) { + LOG(WARNING) << "Failed list files for dir=" << wal_path << ", st=" << st.to_string(); + return st; + } + for (const auto& db_id : dbs) { + if (db_id.is_file) { + continue; + } + std::vector tables; + auto db_path = wal_path + "/" + db_id.file_name; + st = io::global_local_filesystem()->list(db_path, false, &tables, &exists); + if (!st.ok()) { + LOG(WARNING) << "Failed list files for dir=" << db_path << ", st=" << st.to_string(); + return st; + } + for (const auto& table_id : tables) { + if (table_id.is_file) { + continue; + } + std::vector wals; + auto table_path = db_path + "/" + table_id.file_name; + st = io::global_local_filesystem()->list(table_path, false, &wals, &exists); + if (!st.ok()) { + LOG(WARNING) << "Failed list files for dir=" << table_path + << ", st=" << st.to_string(); + return st; + } + if (wals.size() == 0) { + continue; + } + std::vector res; + for (const auto& wal : wals) { + auto wal_file = table_path + "/" + wal.file_name; + res.emplace_back(wal_file); + { + std::lock_guard wrlock(_wal_lock); + int64_t wal_id = std::strtoll(wal.file_name.c_str(), NULL, 10); + _wal_path_map.emplace(wal_id, wal_file); + } + } + st = add_recover_wal(db_id.file_name, table_id.file_name, res); + count += res.size(); + if (!st.ok()) { + LOG(WARNING) << "Failed add replay wal, db=" << db_id.file_name + << ", table=" << table_id.file_name << ", st=" << st.to_string(); + return st; + } + } + } + LOG(INFO) << "Finish list all wals, size:" << count; + return Status::OK(); +} + +Status WalManager::replay() { + do { + if (_stop || _exec_env->master_info() == nullptr) { + break; + } + // port == 0 means not received heartbeat yet + while (_exec_env->master_info()->network_address.port == 0) { + sleep(1); + continue; + } + std::vector replay_tables; + { + std::lock_guard wrlock(_lock); + auto it = _table_map.begin(); + while (it != _table_map.end()) { + if (it->second->size() == 0) { + it = _table_map.erase(it); + } else { + replay_tables.push_back(it->first); + it++; + } + } + } + for (const auto& table_id : replay_tables) { + auto st = _table_map[table_id]->replay_wals(); + if (!st.ok()) { + LOG(WARNING) << "Failed add replay wal on table " << table_id; + } + } + } while (!_stop_background_threads_latch.wait_for( + std::chrono::seconds(config::group_commit_replay_wal_retry_interval_seconds))); + return Status::OK(); +} + +Status WalManager::add_recover_wal(const std::string& db_id, const std::string& table_id, + std::vector wals) { + std::lock_guard wrlock(_lock); + std::shared_ptr table_ptr; + auto it = _table_map.find(table_id); + if (it == _table_map.end()) { + table_ptr = std::make_shared(_exec_env, std::stoll(db_id), std::stoll(table_id)); + _table_map.emplace(table_id, table_ptr); + } else { + table_ptr = it->second; + } + table_ptr->add_wals(wals); + return Status::OK(); +} + +size_t WalManager::get_wal_table_size(const std::string& table_id) { + std::shared_lock rdlock(_lock); + auto it = _table_map.find(table_id); + if (it != _table_map.end()) { + return it->second->size(); + } else { + return 0; + } +} + +Status WalManager::delete_wal(int64_t wal_id) { + { + std::lock_guard wrlock(_wal_lock); + std::string wal_path = _wal_path_map[wal_id]; + RETURN_IF_ERROR(io::global_local_filesystem()->delete_file(wal_path)); + LOG(INFO) << "delete file=" << wal_path; + _wal_path_map.erase(wal_id); + } + return Status::OK(); +} + +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/wal_manager.h b/be/src/olap/wal_manager.h new file mode 100644 index 00000000000000..f5b49f6ddaf2c2 --- /dev/null +++ b/be/src/olap/wal_manager.h @@ -0,0 +1,61 @@ +// 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 "common/config.h" +#include "gen_cpp/FrontendService.h" +#include "gen_cpp/FrontendService_types.h" +#include "gen_cpp/HeartbeatService_types.h" +#include "olap/wal_reader.h" +#include "olap/wal_table.h" +#include "olap/wal_writer.h" +#include "runtime/exec_env.h" +#include "runtime/stream_load/stream_load_context.h" +#include "util/thread.h" + +namespace doris { +class WalManager { + ENABLE_FACTORY_CREATOR(WalManager); + +public: + WalManager(ExecEnv* exec_env, const std::string& wal_dir); + ~WalManager(); + Status delete_wal(int64_t wal_id); + Status init(); + Status scan_wals(const std::string& wal_path); + Status replay(); + Status create_wal_reader(const std::string& wal_path, std::shared_ptr& wal_reader); + Status create_wal_writer(int64_t wal_id, std::shared_ptr& wal_writer); + Status scan(); + size_t get_wal_table_size(const std::string& table_id); + Status add_recover_wal(const std::string& db_id, const std::string& table_id, + std::vector wals); + Status add_wal_path(int64_t db_id, int64_t table_id, int64_t wal_id, const std::string& label); + Status get_wal_path(int64_t wal_id, std::string& wal_path); + void stop(); + +private: + ExecEnv* _exec_env; + std::shared_mutex _lock; + scoped_refptr _replay_thread; + CountDownLatch _stop_background_threads_latch; + std::map> _table_map; + std::vector _wal_dirs; + std::shared_mutex _wal_lock; + std::unordered_map _wal_path_map; + bool _stop = false; +}; +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/wal_reader.cpp b/be/src/olap/wal_reader.cpp new file mode 100644 index 00000000000000..180e3910167900 --- /dev/null +++ b/be/src/olap/wal_reader.cpp @@ -0,0 +1,91 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "olap/wal_reader.h" + +#include "common/status.h" +#include "io/fs/file_reader.h" +#include "io/fs/local_file_system.h" +#include "io/fs/path.h" +#include "util/crc32c.h" +#include "wal_writer.h" + +namespace doris { + +WalReader::WalReader(const std::string& file_name) : _file_name(file_name), _offset(0) {} + +WalReader::~WalReader() {} + +Status WalReader::init() { + RETURN_IF_ERROR(io::global_local_filesystem()->open_file(_file_name, &file_reader)); + return Status::OK(); +} + +Status WalReader::finalize() { + auto st = file_reader->close(); + if (!st.ok()) { + LOG(WARNING) << "fail to close file " << _file_name; + } + return Status::OK(); +} + +Status WalReader::read_block(PBlock& block) { + if (_offset >= file_reader->size()) { + return Status::EndOfFile("end of wal file"); + } + size_t bytes_read = 0; + uint8_t row_len_buf[WalWriter::LENGTH_SIZE]; + RETURN_IF_ERROR( + file_reader->read_at(_offset, {row_len_buf, WalWriter::LENGTH_SIZE}, &bytes_read)); + _offset += WalWriter::LENGTH_SIZE; + size_t block_len; + memcpy(&block_len, row_len_buf, WalWriter::LENGTH_SIZE); + // read block + std::string block_buf; + block_buf.resize(block_len); + RETURN_IF_ERROR(file_reader->read_at(_offset, {block_buf.c_str(), block_len}, &bytes_read)); + _offset += block_len; + RETURN_IF_ERROR(_deserialize(block, block_buf)); + // checksum + uint8_t checksum_len_buf[WalWriter::CHECKSUM_SIZE]; + RETURN_IF_ERROR(file_reader->read_at(_offset, {checksum_len_buf, WalWriter::CHECKSUM_SIZE}, + &bytes_read)); + _offset += WalWriter::CHECKSUM_SIZE; + uint32_t checksum; + memcpy(&checksum, checksum_len_buf, WalWriter::CHECKSUM_SIZE); + RETURN_IF_ERROR(_check_checksum(block_buf.data(), block_len, checksum)); + return Status::OK(); +} + +Status WalReader::_deserialize(PBlock& block, std::string& buf) { + if (UNLIKELY(!block.ParseFromString(buf))) { + return Status::InternalError("failed to deserialize row"); + } + return Status::OK(); +} + +Status WalReader::_check_checksum(const char* binary, size_t size, uint32_t checksum) { + uint32_t computed_checksum = crc32c::Value(binary, size); + if (LIKELY(computed_checksum == checksum)) { + return Status::OK(); + } + return Status::InternalError("checksum failed for wal=" + _file_name + + ", computed checksum=" + std::to_string(computed_checksum) + + ", expected=" + std::to_string(checksum)); +} + +} // namespace doris diff --git a/be/src/olap/wal_reader.h b/be/src/olap/wal_reader.h new file mode 100644 index 00000000000000..825d11fae3016b --- /dev/null +++ b/be/src/olap/wal_reader.h @@ -0,0 +1,45 @@ +// 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 "common/status.h" +#include "gen_cpp/internal_service.pb.h" +#include "io/fs/file_reader_writer_fwd.h" + +namespace doris { + +class WalReader { +public: + explicit WalReader(const std::string& file_name); + ~WalReader(); + + Status init(); + Status finalize(); + + Status read_block(PBlock& block); + +private: + Status _deserialize(PBlock& block, std::string& buf); + Status _check_checksum(const char* binary, size_t size, uint32_t checksum); + + std::string _file_name; + size_t _offset; + io::FileReaderSPtr file_reader; +}; + +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/wal_table.cpp b/be/src/olap/wal_table.cpp new file mode 100644 index 00000000000000..40b48ad7736b7c --- /dev/null +++ b/be/src/olap/wal_table.cpp @@ -0,0 +1,265 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "olap/wal_table.h" + +#include +#include +#include +#include +#include + +#include "evhttp.h" +#include "http/action/stream_load.h" +#include "http/ev_http_server.h" +#include "http/http_common.h" +#include "http/http_headers.h" +#include "http/utils.h" +#include "io/fs/local_file_system.h" +#include "olap/wal_manager.h" +#include "runtime/client_cache.h" +#include "runtime/fragment_mgr.h" +#include "runtime/plan_fragment_executor.h" +#include "util/path_util.h" +#include "util/thrift_rpc_helper.h" +#include "vec/exec/format/wal/wal_reader.h" + +namespace doris { + +WalTable::WalTable(ExecEnv* exec_env, int64_t db_id, int64_t table_id) + : _exec_env(exec_env), _db_id(db_id), _table_id(table_id) {} +WalTable::~WalTable() {} + +#ifdef BE_TEST +std::string k_request_line; +#endif + +void WalTable::add_wals(std::vector wals) { + std::lock_guard lock(_replay_wal_lock); + for (const auto& wal : wals) { + LOG(INFO) << "add replay wal " << wal; + _replay_wal_map.emplace(wal, replay_wal_info {0, UnixMillis(), false}); + } +} +Status WalTable::replay_wals() { + std::vector need_replay_wals; + { + std::lock_guard lock(_replay_wal_lock); + if (_replay_wal_map.empty()) { + return Status::OK(); + } + VLOG_DEBUG << "Start replay wals for db=" << _db_id << ", table=" << _table_id + << ", wal size=" << _replay_wal_map.size(); + for (auto& [wal, info] : _replay_wal_map) { + auto& [retry_num, start_ts, replaying] = info; + if (replaying) { + continue; + } + if (retry_num >= config::group_commit_replay_wal_retry_num) { + LOG(WARNING) << "All replay wal failed, db=" << _db_id << ", table=" << _table_id + << ", wal=" << wal + << ", retry_num=" << config::group_commit_replay_wal_retry_num; + std::string rename_path = get_tmp_path(wal); + LOG(INFO) << "rename wal from " << wal << " to " << rename_path; + std::rename(wal.c_str(), rename_path.c_str()); + _replay_wal_map.erase(wal); + continue; + } + if (need_replay(info)) { + replaying = true; + need_replay_wals.push_back(wal); + } + } + std::sort(need_replay_wals.begin(), need_replay_wals.end()); + } + for (const auto& wal : need_replay_wals) { + auto st = replay_wal_internal(wal); + if (!st.ok()) { + std::lock_guard lock(_replay_wal_lock); + auto it = _replay_wal_map.find(wal); + if (it != _replay_wal_map.end()) { + auto& [retry_num, start_time, replaying] = it->second; + replaying = false; + } + LOG(WARNING) << "failed replay wal, drop this round, db=" << _db_id + << ", table=" << _table_id << ", wal=" << wal << ", st=" << st.to_string(); + break; + } + VLOG_NOTICE << "replay wal, db=" << _db_id << ", table=" << _table_id << ", label=" << wal + << ", st=" << st.to_string(); + } + return Status::OK(); +} + +std::string WalTable::get_tmp_path(const std::string wal) { + std::vector path_element; + doris::vectorized::WalReader::string_split(wal, "/", path_element); + std::stringstream ss; + int index = 0; + while (index < path_element.size() - 3) { + ss << path_element[index] << "/"; + index++; + } + ss << "tmp/"; + while (index < path_element.size()) { + if (index != path_element.size() - 1) { + ss << path_element[index] << "_"; + } else { + ss << path_element[index]; + } + index++; + } + return ss.str(); +} + +bool WalTable::need_replay(const doris::WalTable::replay_wal_info& info) { +#ifndef BE_TEST + auto& [retry_num, start_ts, replaying] = info; + auto replay_interval = + pow(2, retry_num) * config::group_commit_replay_wal_retry_interval_seconds * 1000; + return UnixMillis() - start_ts >= replay_interval; +#else + return true; +#endif +} + +Status WalTable::replay_wal_internal(const std::string& wal) { + LOG(INFO) << "Start replay wal for db=" << _db_id << ", table=" << _table_id << ", wal=" << wal; + // start a new stream load + { + std::lock_guard lock(_replay_wal_lock); + auto it = _replay_wal_map.find(wal); + if (it != _replay_wal_map.end()) { + auto& [retry_num, start_time, replaying] = it->second; + ++retry_num; + replaying = true; + } else { + LOG(WARNING) << "can not find wal in stream load replay map. db=" << _db_id + << ", table=" << _table_id << ", wal=" << wal; + return Status::OK(); + } + } + auto pair = get_wal_info(wal); + auto wal_id = pair.first; + auto label = pair.second; + RETURN_IF_ERROR(send_request(wal_id, wal, label)); + return Status::OK(); +} + +std::pair WalTable::get_wal_info(const std::string& wal) { + std::vector path_element; + doris::vectorized::WalReader::string_split(wal, "/", path_element); + auto pos = path_element[path_element.size() - 1].find("_"); + int64_t wal_id = + std::strtoll(path_element[path_element.size() - 1].substr(0, pos).c_str(), NULL, 10); + auto label = path_element[path_element.size() - 1].substr(pos + 1); + return std::make_pair(wal_id, label); +} + +void http_request_done(struct evhttp_request* req, void* arg) { + event_base_loopbreak((struct event_base*)arg); +} + +Status WalTable::send_request(int64_t wal_id, const std::string& wal, const std::string& label) { +#ifndef BE_TEST + struct event_base* base = nullptr; + struct evhttp_connection* conn = nullptr; + struct evhttp_request* req = nullptr; + event_init(); + base = event_base_new(); + conn = evhttp_connection_new("127.0.0.1", doris::config::webserver_port); + evhttp_connection_set_base(conn, base); + req = evhttp_request_new(http_request_done, base); + evhttp_add_header(req->output_headers, HTTP_LABEL_KEY.c_str(), label.c_str()); + evhttp_add_header(req->output_headers, HTTP_AUTH_CODE.c_str(), std::to_string(wal_id).c_str()); + evhttp_add_header(req->output_headers, HTTP_WAL_ID_KY.c_str(), std::to_string(wal_id).c_str()); + std::stringstream ss; + ss << "insert into " << std::to_string(_table_id) << " WITH LABEL " << label + << " select * from " + "http_stream(\"format\" = \"wal\", \"table_id\" = \"" + << std::to_string(_table_id) << "\")"; + evhttp_add_header(req->output_headers, HTTP_SQL.c_str(), ss.str().c_str()); + evbuffer* output = evhttp_request_get_output_buffer(req); + evbuffer_add_printf(output, "replay wal %s", std::to_string(wal_id).c_str()); + + evhttp_make_request(conn, req, EVHTTP_REQ_PUT, "/api/_http_stream"); + evhttp_connection_set_timeout(req->evcon, 300); + + event_base_dispatch(base); + evhttp_connection_free(conn); + event_base_free(base); + +#endif + bool retry = false; + std::string status; + std::string msg; + std::stringstream out; + rapidjson::Document doc; +#ifndef BE_TEST + size_t len = 0; + auto input = evhttp_request_get_input_buffer(req); + char* request_line = evbuffer_readln(input, &len, EVBUFFER_EOL_CRLF); + while (request_line != nullptr) { + std::string s(request_line); + out << request_line; + request_line = evbuffer_readln(input, &len, EVBUFFER_EOL_CRLF); + } +#else + out << k_request_line; +#endif + auto out_str = out.str(); + if (!out_str.empty()) { + doc.Parse(out.str().c_str()); + status = std::string(doc["Status"].GetString()); + msg = std::string(doc["Message"].GetString()); + LOG(INFO) << "replay wal " << wal_id << " status:" << status << ",msg:" << msg; + if (status.find("Fail") != status.npos) { + if (msg.find("Label") != msg.npos && msg.find("has already been used") != msg.npos) { + retry = false; + } else { + retry = true; + } + } else { + retry = false; + } + } else { + retry = true; + } + if (retry) { + LOG(INFO) << "fail to replay wal =" << wal << ",status:" << status << ",msg:" << msg; + std::lock_guard lock(_replay_wal_lock); + auto it = _replay_wal_map.find(wal); + if (it != _replay_wal_map.end()) { + auto& [retry_num, start_time, replaying] = it->second; + replaying = false; + } else { + _replay_wal_map.emplace(wal, replay_wal_info {0, UnixMillis(), false}); + } + } else { + LOG(INFO) << "success to replay wal =" << wal << ",status:" << status << ",msg:" << msg; + RETURN_IF_ERROR(_exec_env->wal_mgr()->delete_wal(wal_id)); + std::lock_guard lock(_replay_wal_lock); + _replay_wal_map.erase(wal); + } + return Status::OK(); +} + +size_t WalTable::size() { + std::lock_guard lock(_replay_wal_lock); + return _replay_wal_map.size(); +} +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/wal_table.h b/be/src/olap/wal_table.h new file mode 100644 index 00000000000000..2dd63240d3547e --- /dev/null +++ b/be/src/olap/wal_table.h @@ -0,0 +1,58 @@ +// 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 "common/status.h" +#include "gen_cpp/FrontendService.h" +#include "gen_cpp/FrontendService_types.h" +#include "gen_cpp/HeartbeatService_types.h" +#include "runtime/exec_env.h" +#include "runtime/stream_load/stream_load_context.h" +namespace doris { +class WalTable { +public: + WalTable(ExecEnv* exec_env, int64_t db_id, int64_t table_id); + ~WalTable(); + // + using replay_wal_info = std::tuple; + // used when be start and there are wals need to do recovery + void add_wals(std::vector wals); + Status replay_wals(); + size_t size(); + +private: + std::pair get_wal_info(const std::string& wal); + std::string get_tmp_path(const std::string wal); + Status send_request(int64_t wal_id, const std::string& wal, const std::string& label); + +private: + ExecEnv* _exec_env; + int64_t _db_id; + int64_t _table_id; + std::string _relay = "relay"; + std::string _split = "_"; + mutable std::mutex _replay_wal_lock; + // key is wal_id + std::map _replay_wal_map; + bool need_replay(const replay_wal_info& info); + Status replay_wal_internal(const std::string& wal); +}; +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/wal_writer.cpp b/be/src/olap/wal_writer.cpp new file mode 100644 index 00000000000000..7cd427453bb361 --- /dev/null +++ b/be/src/olap/wal_writer.cpp @@ -0,0 +1,75 @@ +// 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 "olap/wal_writer.h" + +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" +#include "io/fs/path.h" +#include "olap/storage_engine.h" +#include "util/crc32c.h" + +namespace doris { + +WalWriter::WalWriter(const std::string& file_name) : _file_name(file_name) {} + +WalWriter::~WalWriter() {} + +Status WalWriter::init() { + _batch = config::group_commit_sync_wal_batch; + RETURN_IF_ERROR(io::global_local_filesystem()->create_file(_file_name, &_file_writer)); + return Status::OK(); +} + +Status WalWriter::finalize() { + auto st = _file_writer->close(); + if (!st.ok()) { + LOG(WARNING) << "fail to close file " << _file_name; + } + return Status::OK(); +} + +Status WalWriter::append_blocks(const PBlockArray& blocks) { + size_t total_size = 0; + for (const auto& block : blocks) { + total_size += LENGTH_SIZE + block->ByteSizeLong() + CHECKSUM_SIZE; + } + std::string binary(total_size, '\0'); + char* row_binary = binary.data(); + size_t offset = 0; + for (const auto& block : blocks) { + unsigned long row_length = block->GetCachedSize(); + memcpy(row_binary + offset, &row_length, LENGTH_SIZE); + offset += LENGTH_SIZE; + memcpy(row_binary + offset, block->SerializeAsString().data(), row_length); + offset += row_length; + uint32_t checksum = crc32c::Value(block->SerializeAsString().data(), row_length); + memcpy(row_binary + offset, &checksum, CHECKSUM_SIZE); + offset += CHECKSUM_SIZE; + } + DCHECK(offset == total_size); + // write rows + RETURN_IF_ERROR(_file_writer->append({row_binary, offset})); + _count++; + if (_count % _batch == 0) { + //todo sync data + //LOG(INFO) << "count=" << count << ",do sync"; + } + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/olap/wal_writer.h b/be/src/olap/wal_writer.h new file mode 100644 index 00000000000000..12fd84f258fb7d --- /dev/null +++ b/be/src/olap/wal_writer.h @@ -0,0 +1,49 @@ +// 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 "common/status.h" +#include "gen_cpp/internal_service.pb.h" +#include "io/fs/file_reader_writer_fwd.h" + +namespace doris { + +using PBlockArray = std::vector; + +class WalWriter { +public: + explicit WalWriter(const std::string& file_name); + ~WalWriter(); + + Status init(); + Status finalize(); + + Status append_blocks(const PBlockArray& blocks); + + std::string file_name() { return _file_name; }; + static const int64_t LENGTH_SIZE = 8; + static const int64_t CHECKSUM_SIZE = 4; + +private: + std::string _file_name; + io::FileWriterPtr _file_writer; + int64_t _count; + int64_t _batch; +}; + +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp index 49af73335f010d..2f39962a17ceea 100644 --- a/be/src/pipeline/exec/olap_scan_operator.cpp +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -359,7 +359,8 @@ inline std::string push_down_agg_to_string(const TPushAggOp::type& op) { Status OlapScanLocalState::_build_key_ranges_and_filters() { auto& p = _parent->cast(); - if (p._push_down_agg_type == TPushAggOp::NONE) { + if (p._push_down_agg_type == TPushAggOp::NONE || + p._push_down_agg_type == TPushAggOp::COUNT_ON_INDEX) { const std::vector& column_names = p._olap_scan_node.key_column_name; const std::vector& column_types = p._olap_scan_node.key_column_type; DCHECK(column_types.size() == column_names.size()); diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp index 8a9f8119d2879c..9244a435856966 100644 --- a/be/src/pipeline/pipeline_x/operator.cpp +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -170,10 +170,10 @@ Status OperatorXBase::do_projections(RuntimeState* state, vectorized::Block* ori if (rows != 0) { auto& mutable_columns = mutable_block.mutable_columns(); - DCHECK(mutable_columns.size() == _projections.size()); + DCHECK(mutable_columns.size() == local_state->_projections.size()); for (int i = 0; i < mutable_columns.size(); ++i) { auto result_column_id = -1; - RETURN_IF_ERROR(_projections[i]->execute(origin_block, &result_column_id)); + RETURN_IF_ERROR(local_state->_projections[i]->execute(origin_block, &result_column_id)); auto column_ptr = origin_block->get_by_position(result_column_id) .column->convert_to_full_column_if_const(); //TODO: this is a quick fix, we need a new function like "change_to_nullable" to do it diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index 37a117b2bf658d..c1675882346ba7 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -131,12 +131,6 @@ void PipelineXFragmentContext::cancel(const PPlanFragmentCancelReason& reason, stream_load_ctx->pipe->cancel(msg); } - // must close stream_mgr to avoid dead lock in Exchange Node - FOR_EACH_RUNTIME_STATE( - runtime_state->set_is_cancelled(true, msg); - runtime_state->set_process_status(_query_ctx->exec_status()); - _exec_env->vstream_mgr()->cancel(runtime_state->fragment_instance_id());) - // Cancel the result queue manager used by spark doris connector // TODO pipeline incomp // _exec_env->result_queue_mgr()->update_queue_status(id, Status::Aborted(msg)); diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index ed59a81916a581..9fffd859ba3617 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -99,6 +99,7 @@ class SegmentLoader; class LookupConnectionCache; class RowCache; class CacheManager; +class WalManager; inline bool k_doris_exit = false; @@ -209,6 +210,7 @@ class ExecEnv { doris::vectorized::ScannerScheduler* scanner_scheduler() { return _scanner_scheduler; } FileMetaCache* file_meta_cache() { return _file_meta_cache; } MemTableMemoryLimiter* memtable_memory_limiter() { return _memtable_memory_limiter.get(); } + WalManager* wal_mgr() { return _wal_manager.get(); } #ifdef BE_TEST void set_ready() { this->_s_ready = true; } void set_not_ready() { this->_s_ready = false; } @@ -344,6 +346,7 @@ class ExecEnv { std::unique_ptr _memtable_memory_limiter; std::unique_ptr _load_stream_stub_pool; std::unique_ptr _delta_writer_v2_pool; + std::shared_ptr _wal_manager; std::mutex _frontends_lock; std::map _frontends; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index ebdf0dd4fcc5d7..59c2b96d537430 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -48,6 +48,7 @@ #include "olap/schema_cache.h" #include "olap/segment_loader.h" #include "olap/storage_engine.h" +#include "olap/wal_manager.h" #include "pipeline/task_queue.h" #include "pipeline/task_scheduler.h" #include "runtime/block_spill_manager.h" @@ -210,6 +211,7 @@ Status ExecEnv::_init(const std::vector& store_paths, _memtable_memory_limiter = std::make_unique(); _load_stream_stub_pool = std::make_unique(); _delta_writer_v2_pool = std::make_unique(); + _wal_manager = WalManager::create_shared(this, config::group_commit_replay_wal_dir); _backend_client_cache->init_metrics("backend"); _frontend_client_cache->init_metrics("frontend"); @@ -232,6 +234,7 @@ Status ExecEnv::_init(const std::vector& store_paths, RETURN_IF_ERROR(_memtable_memory_limiter->init(MemInfo::mem_limit())); RETURN_IF_ERROR(_load_channel_mgr->init(MemInfo::mem_limit())); + RETURN_IF_ERROR(_wal_manager->init()); _heartbeat_flags = new HeartbeatFlags(); _register_metrics(); @@ -526,6 +529,8 @@ void ExecEnv::destroy() { // Memory barrier to prevent other threads from accessing destructed resources _s_ready = false; + SAFE_STOP(_wal_manager); + _wal_manager.reset(); SAFE_STOP(_tablet_schema_cache); SAFE_STOP(_load_channel_mgr); SAFE_STOP(_scanner_scheduler); diff --git a/be/src/runtime/group_commit_mgr.cpp b/be/src/runtime/group_commit_mgr.cpp index 94c0dba30a0d3f..f039cf34e8b478 100644 --- a/be/src/runtime/group_commit_mgr.cpp +++ b/be/src/runtime/group_commit_mgr.cpp @@ -27,6 +27,7 @@ #include "common/object_pool.h" #include "exec/data_sink.h" #include "io/fs/stream_load_pipe.h" +#include "olap/wal_manager.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_state.h" @@ -34,11 +35,11 @@ #include "runtime/stream_load/stream_load_context.h" #include "util/thrift_rpc_helper.h" #include "vec/exec/scan/new_file_scan_node.h" +#include "vec/sink/group_commit_block_sink.h" namespace doris { class TPlan; -class FragmentExecState; Status LoadBlockQueue::add_block(std::shared_ptr block) { DCHECK(block->get_schema_version() == schema_version); @@ -185,16 +186,17 @@ Status GroupCommitTable::get_first_block_load_queue( Status GroupCommitTable::_create_group_commit_load( int64_t table_id, std::shared_ptr& load_block_queue) { TStreamLoadPutRequest request; - std::stringstream ss; - ss << "insert into " << table_id << " select * from group_commit(\"table_id\"=\"" << table_id - << "\")"; - request.__set_load_sql(ss.str()); UniqueId load_id = UniqueId::gen_uid(); TUniqueId tload_id; tload_id.__set_hi(load_id.hi); tload_id.__set_lo(load_id.lo); + std::regex reg("-"); + std::string label = "group_commit_" + std::regex_replace(load_id.to_string(), reg, "_"); + std::stringstream ss; + ss << "insert into " << table_id << " WITH LABEL " << label + << " select * from group_commit(\"table_id\"=\"" << table_id << "\")"; + request.__set_load_sql(ss.str()); request.__set_loadId(tload_id); - std::string label = "group_commit_" + load_id.to_string(); request.__set_label(label); request.__set_token("group_commit"); // this is a fake, fe not check it now request.__set_max_filter_ratio(1.0); @@ -243,6 +245,7 @@ Status GroupCommitTable::_create_group_commit_load( std::unique_lock l(_lock); _load_block_queues.emplace(instance_id, load_block_queue); } + params.__set_import_label(label); st = _exec_plan_fragment(_db_id, table_id, label, txn_id, is_pipeline, params, pipeline_params); if (!st.ok()) { _finish_group_commit_load(_db_id, table_id, label, txn_id, instance_id, st, true, nullptr); @@ -256,7 +259,7 @@ Status GroupCommitTable::_finish_group_commit_load(int64_t db_id, int64_t table_ bool prepare_failed, RuntimeState* state) { { std::lock_guard l(_lock); - if (prepare_failed) { + if (prepare_failed || !status.ok()) { auto it = _load_block_queues.find(instance_id); if (it != _load_block_queues.end()) { it->second->cancel(status); @@ -308,9 +311,27 @@ Status GroupCommitTable::_finish_group_commit_load(int64_t db_id, int64_t table_ << ", instance_id=" << print_id(instance_id) << ", executor status=" << status.to_string() << ", request commit status=" << st.to_string(); + if (!prepare_failed) { + RETURN_IF_ERROR(_exec_env->wal_mgr()->add_wal_path(_db_id, table_id, txn_id, label)); + std::string wal_path; + RETURN_IF_ERROR(_exec_env->wal_mgr()->get_wal_path(txn_id, wal_path)); + RETURN_IF_ERROR(_exec_env->wal_mgr()->add_recover_wal( + std::to_string(db_id), std::to_string(table_id), + std::vector {wal_path})); + } return st; } // TODO handle execute and commit error + if (!prepare_failed && !result_status.ok()) { + RETURN_IF_ERROR(_exec_env->wal_mgr()->add_wal_path(_db_id, table_id, txn_id, label)); + std::string wal_path; + RETURN_IF_ERROR(_exec_env->wal_mgr()->get_wal_path(txn_id, wal_path)); + RETURN_IF_ERROR(_exec_env->wal_mgr()->add_recover_wal(std::to_string(db_id), + std::to_string(table_id), + std::vector {wal_path})); + } else { + RETURN_IF_ERROR(_exec_env->wal_mgr()->delete_wal(txn_id)); + } std::stringstream ss; ss << "finish group commit, db_id=" << db_id << ", table_id=" << table_id << ", label=" << label << ", txn_id=" << txn_id << ", instance_id=" << print_id(instance_id); @@ -490,6 +511,130 @@ Status GroupCommitMgr::_append_row(std::shared_ptr pipe, return Status::OK(); } +Status GroupCommitMgr::group_commit_stream_load(std::shared_ptr ctx) { + return _insert_into_thread_pool->submit_func([ctx, this] { + Status st = _group_commit_stream_load(ctx); + if (!st.ok()) { + ctx->promise.set_value(st); + } + }); +} + +Status GroupCommitMgr::_group_commit_stream_load(std::shared_ptr ctx) { + auto& fragment_params = ctx->put_result.params; + auto& tdesc_tbl = fragment_params.desc_tbl; + DCHECK(fragment_params.params.per_node_scan_ranges.size() == 1); + DCHECK(fragment_params.params.per_node_scan_ranges.begin()->second.size() == 1); + auto& tscan_range_params = fragment_params.params.per_node_scan_ranges.begin()->second.at(0); + auto& nodes = fragment_params.fragment.plan.nodes; + DCHECK(nodes.size() > 0); + auto& plan_node = nodes.at(0); + + std::vector> future_blocks; + { + std::shared_ptr load_block_queue; + // 1. FileScanNode consumes data from the pipe. + std::unique_ptr runtime_state = RuntimeState::create_unique(); + TUniqueId load_id; + load_id.hi = ctx->id.hi; + load_id.lo = ctx->id.lo; + TQueryOptions query_options; + query_options.query_type = TQueryType::LOAD; + TQueryGlobals query_globals; + runtime_state->init(load_id, query_options, query_globals, _exec_env); + runtime_state->set_query_mem_tracker(std::make_shared( + MemTrackerLimiter::Type::LOAD, fmt::format("Load#Id={}", ctx->id.to_string()), -1)); + DescriptorTbl* desc_tbl = nullptr; + RETURN_IF_ERROR(DescriptorTbl::create(runtime_state->obj_pool(), tdesc_tbl, &desc_tbl)); + runtime_state->set_desc_tbl(desc_tbl); + auto file_scan_node = + vectorized::NewFileScanNode(runtime_state->obj_pool(), plan_node, *desc_tbl); + Status status = Status::OK(); + auto sink = stream_load::GroupCommitBlockSink( + runtime_state->obj_pool(), file_scan_node.row_desc(), + fragment_params.fragment.output_exprs, &status); + std::unique_ptr> close_scan_node_func((int*)0x01, [&](int*) { + if (load_block_queue != nullptr) { + load_block_queue->remove_load_id(load_id); + } + file_scan_node.close(runtime_state.get()); + sink.close(runtime_state.get(), status); + }); + RETURN_IF_ERROR(file_scan_node.init(plan_node, runtime_state.get())); + RETURN_IF_ERROR(file_scan_node.prepare(runtime_state.get())); + std::vector params_vector; + params_vector.emplace_back(tscan_range_params); + file_scan_node.set_scan_ranges(params_vector); + RETURN_IF_ERROR(file_scan_node.open(runtime_state.get())); + + RETURN_IF_ERROR(status); + RETURN_IF_ERROR(sink.init(fragment_params.fragment.output_sink)); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION(sink.prepare(runtime_state.get())); + RETURN_IF_ERROR(sink.open(runtime_state.get())); + + // 2. Put the block into block queue. + std::unique_ptr _block = + doris::vectorized::Block::create_unique(); + bool first = true; + bool eof = false; + while (!eof) { + // TODO what to do if scan one block error + RETURN_IF_ERROR(file_scan_node.get_next(runtime_state.get(), _block.get(), &eof)); + RETURN_IF_ERROR(sink.send(runtime_state.get(), _block.get())); + std::shared_ptr future_block = + std::make_shared(); + future_block->swap(*(_block.get())); + future_block->set_info(ctx->schema_version, load_id, first, eof); + // TODO what to do if add one block error + if (load_block_queue == nullptr) { + RETURN_IF_ERROR(_get_first_block_load_queue(ctx->db_id, ctx->table_id, future_block, + load_block_queue)); + ctx->label = load_block_queue->label; + ctx->txn_id = load_block_queue->txn_id; + } + RETURN_IF_ERROR(load_block_queue->add_block(future_block)); + if (future_block->rows() > 0) { + future_blocks.emplace_back(future_block); + } + first = false; + } + ctx->number_unselected_rows = runtime_state->num_rows_load_unselected(); + ctx->number_filtered_rows = runtime_state->num_rows_load_filtered(); + ctx->error_url = runtime_state->get_error_log_file_path(); + if (!runtime_state->get_error_log_file_path().empty()) { + LOG(INFO) << "id=" << print_id(load_id) + << ", url=" << runtime_state->get_error_log_file_path() + << ", load rows=" << runtime_state->num_rows_load_total() + << ", filter rows=" << runtime_state->num_rows_load_filtered() + << ", unselect rows=" << runtime_state->num_rows_load_unselected() + << ", success rows=" << runtime_state->num_rows_load_success(); + } + } + + int64_t total_rows = 0; + int64_t loaded_rows = 0; + // 3. wait to wal + for (const auto& future_block : future_blocks) { + std::unique_lock l(*(future_block->lock)); + if (!future_block->is_handled()) { + future_block->cv->wait(l); + } + // future_block->get_status() + total_rows += future_block->get_total_rows(); + loaded_rows += future_block->get_loaded_rows(); + } + ctx->number_total_rows = total_rows + ctx->number_unselected_rows + ctx->number_filtered_rows; + ctx->number_loaded_rows = loaded_rows; + ctx->number_filtered_rows += total_rows - ctx->number_loaded_rows; + ctx->promise.set_value(Status::OK()); + VLOG_DEBUG << "finish read all block of pipe=" << ctx->id.to_string() + << ", total rows=" << ctx->number_total_rows + << ", loaded rows=" << ctx->number_loaded_rows + << ", filtered rows=" << ctx->number_filtered_rows + << ", unselected rows=" << ctx->number_unselected_rows; + return Status::OK(); +} + Status GroupCommitMgr::_get_first_block_load_queue( int64_t db_id, int64_t table_id, std::shared_ptr block, std::shared_ptr& load_block_queue) { diff --git a/be/src/runtime/group_commit_mgr.h b/be/src/runtime/group_commit_mgr.h index 1d124009d1987a..01a0905c404f33 100644 --- a/be/src/runtime/group_commit_mgr.h +++ b/be/src/runtime/group_commit_mgr.h @@ -35,6 +35,7 @@ class TUniqueId; class TExecPlanFragmentParams; class ObjectPool; class RuntimeState; +class StreamLoadContext; class StreamLoadPipe; class LoadBlockQueue { @@ -118,6 +119,9 @@ class GroupCommitMgr { const PGroupCommitInsertRequest* request, PGroupCommitInsertResponse* response); + // stream load + Status group_commit_stream_load(std::shared_ptr ctx); + // used when init group_commit_scan_node Status get_load_block_queue(int64_t table_id, const TUniqueId& instance_id, std::shared_ptr& load_block_queue); @@ -126,6 +130,8 @@ class GroupCommitMgr { // used by insert into Status _append_row(std::shared_ptr pipe, const PGroupCommitInsertRequest* request); + // used by stream load + Status _group_commit_stream_load(std::shared_ptr ctx); Status _get_first_block_load_queue(int64_t db_id, int64_t table_id, std::shared_ptr block, std::shared_ptr& load_block_queue); diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index 18ef1871ecc8d1..59a89b44af3a50 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -156,6 +156,9 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request) { if (request.__isset.load_job_id) { _runtime_state->set_load_job_id(request.load_job_id); } + if (request.__isset.wal_id) { + _runtime_state->set_wal_id(request.wal_id); + } if (request.query_options.__isset.is_report_success) { _is_report_success = request.query_options.is_report_success; diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 5275a21c64d026..87f0f45b6aaf18 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -229,6 +229,12 @@ class RuntimeState { const std::string& db_name() { return _db_name; } + void set_wal_id(int64_t wal_id) { _wal_id = wal_id; } + + int64_t wal_id() { return _wal_id; } + + const std::string& import_label() { return _import_label; } + const std::string& load_dir() const { return _load_dir; } void set_load_job_id(int64_t job_id) { _load_job_id = job_id; } @@ -547,6 +553,7 @@ class RuntimeState { std::string _db_name; std::string _load_dir; int64_t _load_job_id; + int64_t _wal_id = -1; // mini load int64_t _normal_row_number; diff --git a/be/src/runtime/stream_load/stream_load_context.cpp b/be/src/runtime/stream_load/stream_load_context.cpp index f381ba097db1a5..0cd1f0e3d59122 100644 --- a/be/src/runtime/stream_load/stream_load_context.cpp +++ b/be/src/runtime/stream_load/stream_load_context.cpp @@ -50,9 +50,14 @@ std::string StreamLoadContext::to_json() const { writer.Key("Comment"); writer.String(load_comment.c_str()); - writer.Key("TwoPhaseCommit"); - std::string need_two_phase_commit = two_phase_commit ? "true" : "false"; - writer.String(need_two_phase_commit.c_str()); + if (!group_commit) { + writer.Key("TwoPhaseCommit"); + std::string need_two_phase_commit = two_phase_commit ? "true" : "false"; + writer.String(need_two_phase_commit.c_str()); + } else { + writer.Key("GroupCommit"); + writer.Bool(true); + } // status writer.Key("Status"); @@ -92,16 +97,20 @@ std::string StreamLoadContext::to_json() const { writer.Int64(receive_bytes); writer.Key("LoadTimeMs"); writer.Int64(load_cost_millis); - writer.Key("BeginTxnTimeMs"); - writer.Int64(begin_txn_cost_nanos / 1000000); + if (!group_commit) { + writer.Key("BeginTxnTimeMs"); + writer.Int64(begin_txn_cost_nanos / 1000000); + } writer.Key("StreamLoadPutTimeMs"); writer.Int64(stream_load_put_cost_nanos / 1000000); writer.Key("ReadDataTimeMs"); writer.Int64(read_data_cost_nanos / 1000000); writer.Key("WriteDataTimeMs"); writer.Int(write_data_cost_nanos / 1000000); - writer.Key("CommitAndPublishTimeMs"); - writer.Int64(commit_and_publish_txn_cost_nanos / 1000000); + if (!group_commit) { + writer.Key("CommitAndPublishTimeMs"); + writer.Int64(commit_and_publish_txn_cost_nanos / 1000000); + } if (!error_url.empty()) { writer.Key("ErrorURL"); diff --git a/be/src/runtime/stream_load/stream_load_context.h b/be/src/runtime/stream_load/stream_load_context.h index b4be684fbb6a7b..ab8cc6be044961 100644 --- a/be/src/runtime/stream_load/stream_load_context.h +++ b/be/src/runtime/stream_load/stream_load_context.h @@ -134,7 +134,10 @@ class StreamLoadContext { std::string db; int64_t db_id = -1; + int64_t wal_id = -1; std::string table; + int64_t table_id = -1; + int64_t schema_version = -1; std::string label; // optional std::string sub_label; @@ -173,6 +176,7 @@ class StreamLoadContext { bool use_streaming = false; TFileFormatType::type format = TFileFormatType::FORMAT_CSV_PLAIN; TFileCompressType::type compress_type = TFileCompressType::UNKNOWN; + bool group_commit = false; std::shared_ptr body_sink; std::shared_ptr pipe; diff --git a/be/src/udf/udf.cpp b/be/src/udf/udf.cpp index 37c8e72d170239..ecabc905ab8335 100644 --- a/be/src/udf/udf.cpp +++ b/be/src/udf/udf.cpp @@ -56,6 +56,7 @@ std::unique_ptr FunctionContext::clone() { auto new_context = create_context(_state, _return_type, _arg_types); new_context->_constant_cols = _constant_cols; new_context->_fragment_local_fn_state = _fragment_local_fn_state; + new_context->_check_overflow_for_decimal = _check_overflow_for_decimal; return new_context; } diff --git a/be/src/util/load_util.cpp b/be/src/util/load_util.cpp index 1277132378b85a..1a0dff2f904ff6 100644 --- a/be/src/util/load_util.cpp +++ b/be/src/util/load_util.cpp @@ -67,6 +67,8 @@ void LoadUtil::parse_format(const std::string& format_str, const std::string& co *format_type = TFileFormatType::FORMAT_PARQUET; } else if (iequal(format_str, "ORC")) { *format_type = TFileFormatType::FORMAT_ORC; + } else if (iequal(format_str, "WAL")) { + *format_type = TFileFormatType::FORMAT_WAL; } return; } @@ -82,6 +84,7 @@ bool LoadUtil::is_format_support_streaming(TFileFormatType::type format) { case TFileFormatType::FORMAT_CSV_LZO: case TFileFormatType::FORMAT_CSV_LZOP: case TFileFormatType::FORMAT_JSON: + case TFileFormatType::FORMAT_WAL: return true; default: return false; diff --git a/be/src/vec/data_types/serde/data_type_array_serde.cpp b/be/src/vec/data_types/serde/data_type_array_serde.cpp index b73c2ef9a5dfe2..25b628e20a612f 100644 --- a/be/src/vec/data_types/serde/data_type_array_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_array_serde.cpp @@ -322,5 +322,33 @@ Status DataTypeArraySerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeArraySerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, std::vector& buffer_list) const { + orc::ListVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + cur_batch->offsets[0] = 0; + + const ColumnArray& array_col = assert_cast(column); + const IColumn& nested_column = array_col.get_data(); + auto& offsets = array_col.get_offsets(); + + cur_batch->elements->resize(nested_column.size()); + for (size_t row_id = start; row_id < end; row_id++) { + size_t offset = offsets[row_id - 1]; + size_t next_offset = offsets[row_id]; + + if (cur_batch->notNull[row_id] == 1) { + nested_serde->write_column_to_orc(nested_column, nullptr, cur_batch->elements.get(), + offset, next_offset, buffer_list); + } + + cur_batch->offsets[row_id + 1] = next_offset; + } + cur_batch->elements->numElements = nested_column.size(); + + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_array_serde.h b/be/src/vec/data_types/serde/data_type_array_serde.h index 0f2aab7982132d..e93b1414dc4530 100644 --- a/be/src/vec/data_types/serde/data_type_array_serde.h +++ b/be/src/vec/data_types/serde/data_type_array_serde.h @@ -88,6 +88,10 @@ class DataTypeArraySerDe : public DataTypeSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + void set_return_object_as_string(bool value) override { DataTypeSerDe::set_return_object_as_string(value); nested_serde->set_return_object_as_string(value); diff --git a/be/src/vec/data_types/serde/data_type_bitmap_serde.cpp b/be/src/vec/data_types/serde/data_type_bitmap_serde.cpp index e11bdfcea29172..af50281811ec7b 100644 --- a/be/src/vec/data_types/serde/data_type_bitmap_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_bitmap_serde.cpp @@ -139,5 +139,24 @@ Status DataTypeBitMapSerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeBitMapSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, + std::vector& buffer_list) const { + auto& col_data = assert_cast(column); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + const auto& ele = col_data.get_data_at(row_id); + cur_batch->data[row_id] = const_cast(ele.data); + cur_batch->length[row_id] = ele.size; + } + } + + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_bitmap_serde.h b/be/src/vec/data_types/serde/data_type_bitmap_serde.h index 8e31be880a0190..c53a52caf0e9db 100644 --- a/be/src/vec/data_types/serde/data_type_bitmap_serde.h +++ b/be/src/vec/data_types/serde/data_type_bitmap_serde.h @@ -80,6 +80,10 @@ class DataTypeBitMapSerDe : public DataTypeSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + private: // Bitmap is binary data which is not shown by mysql. template diff --git a/be/src/vec/data_types/serde/data_type_date64_serde.cpp b/be/src/vec/data_types/serde/data_type_date64_serde.cpp index 4e19841054adb0..8c538710c91558 100644 --- a/be/src/vec/data_types/serde/data_type_date64_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_date64_serde.cpp @@ -273,5 +273,49 @@ Status DataTypeDate64SerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeDate64SerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, + std::vector& buffer_list) const { + auto& col_data = static_cast&>(column).get_data(); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); + if (!ptr) { + return Status::InternalError( + "malloc memory error when write largeint column data to orc file."); + } + StringRef bufferRef; + bufferRef.data = ptr; + bufferRef.size = BUFFER_UNIT_SIZE; + size_t offset = 0; + const size_t begin_off = offset; + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 0) { + continue; + } + + int len = binary_cast(col_data[row_id]) + .to_buffer(const_cast(bufferRef.data) + offset); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + cur_batch->length[row_id] = len; + offset += len; + } + size_t data_off = 0; + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; + data_off += cur_batch->length[row_id]; + } + } + + buffer_list.emplace_back(bufferRef); + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_date64_serde.h b/be/src/vec/data_types/serde/data_type_date64_serde.h index 52afdcd65e6d5c..560d1cad0c43e0 100644 --- a/be/src/vec/data_types/serde/data_type_date64_serde.h +++ b/be/src/vec/data_types/serde/data_type_date64_serde.h @@ -64,6 +64,10 @@ class DataTypeDate64SerDe : public DataTypeNumberSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_datetimev2_serde.cpp b/be/src/vec/data_types/serde/data_type_datetimev2_serde.cpp index a5ef23914e53d6..3e7b1b7babdfa9 100644 --- a/be/src/vec/data_types/serde/data_type_datetimev2_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_datetimev2_serde.cpp @@ -137,5 +137,50 @@ Status DataTypeDateTimeV2SerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeDateTimeV2SerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, + int start, int end, + std::vector& buffer_list) const { + auto& col_data = assert_cast&>(column).get_data(); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); + if (!ptr) { + return Status::InternalError( + "malloc memory error when write largeint column data to orc file."); + } + StringRef bufferRef; + bufferRef.data = ptr; + bufferRef.size = BUFFER_UNIT_SIZE; + size_t offset = 0; + const size_t begin_off = offset; + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 0) { + continue; + } + + int len = binary_cast>(col_data[row_id]) + .to_buffer(const_cast(bufferRef.data) + offset, scale); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + cur_batch->length[row_id] = len; + offset += len; + } + + size_t data_off = 0; + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; + data_off += cur_batch->length[row_id]; + } + } + + buffer_list.emplace_back(bufferRef); + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_datetimev2_serde.h b/be/src/vec/data_types/serde/data_type_datetimev2_serde.h index ab2c3a30f60ce5..8cc26a550d282b 100644 --- a/be/src/vec/data_types/serde/data_type_datetimev2_serde.h +++ b/be/src/vec/data_types/serde/data_type_datetimev2_serde.h @@ -73,6 +73,10 @@ class DataTypeDateTimeV2SerDe : public DataTypeNumberSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_datev2_serde.cpp b/be/src/vec/data_types/serde/data_type_datev2_serde.cpp index 0718b8e7985349..663801ffbf7aec 100644 --- a/be/src/vec/data_types/serde/data_type_datev2_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_datev2_serde.cpp @@ -142,5 +142,50 @@ Status DataTypeDateV2SerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeDateV2SerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, + std::vector& buffer_list) const { + auto& col_data = assert_cast&>(column).get_data(); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); + if (!ptr) { + return Status::InternalError( + "malloc memory error when write largeint column data to orc file."); + } + StringRef bufferRef; + bufferRef.data = ptr; + bufferRef.size = BUFFER_UNIT_SIZE; + size_t offset = 0; + const size_t begin_off = offset; + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 0) { + continue; + } + + int len = binary_cast>(col_data[row_id]) + .to_buffer(const_cast(bufferRef.data) + offset); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + cur_batch->length[row_id] = len; + offset += len; + } + + size_t data_off = 0; + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; + data_off += cur_batch->length[row_id]; + } + } + + buffer_list.emplace_back(bufferRef); + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris \ No newline at end of file diff --git a/be/src/vec/data_types/serde/data_type_datev2_serde.h b/be/src/vec/data_types/serde/data_type_datev2_serde.h index 1fa3ee70b46fb0..3610ebb56f897b 100644 --- a/be/src/vec/data_types/serde/data_type_datev2_serde.h +++ b/be/src/vec/data_types/serde/data_type_datev2_serde.h @@ -65,6 +65,10 @@ class DataTypeDateV2SerDe : public DataTypeNumberSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_decimal_serde.cpp b/be/src/vec/data_types/serde/data_type_decimal_serde.cpp index e70e5d4d2caf16..1a7732aad4d0ef 100644 --- a/be/src/vec/data_types/serde/data_type_decimal_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_decimal_serde.cpp @@ -237,6 +237,39 @@ Status DataTypeDecimalSerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +template +Status DataTypeDecimalSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, + int start, int end, + std::vector& buffer_list) const { + auto& col_data = assert_cast&>(column).get_data(); + + if constexpr (IsDecimal128 || IsDecimal128I) { + orc::Decimal128VectorBatch* cur_batch = + dynamic_cast(orc_col_batch); + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + auto& v = col_data[row_id]; + orc::Int128 value(v >> 64, (uint64_t)v); + cur_batch->values[row_id] = value; + } + } + cur_batch->numElements = end - start; + } else { + orc::Decimal64VectorBatch* cur_batch = + dynamic_cast(orc_col_batch); + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + cur_batch->values[row_id] = col_data[row_id]; + } + } + cur_batch->numElements = end - start; + } + return Status::OK(); +} + template class DataTypeDecimalSerDe; template class DataTypeDecimalSerDe; template class DataTypeDecimalSerDe; diff --git a/be/src/vec/data_types/serde/data_type_decimal_serde.h b/be/src/vec/data_types/serde/data_type_decimal_serde.h index d20ff4c4d56420..697f05569a5d94 100644 --- a/be/src/vec/data_types/serde/data_type_decimal_serde.h +++ b/be/src/vec/data_types/serde/data_type_decimal_serde.h @@ -102,6 +102,10 @@ class DataTypeDecimalSerDe : public DataTypeSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_fixedlengthobject_serde.h b/be/src/vec/data_types/serde/data_type_fixedlengthobject_serde.h index 80fee3dcbd98ce..6546281904b2cf 100644 --- a/be/src/vec/data_types/serde/data_type_fixedlengthobject_serde.h +++ b/be/src/vec/data_types/serde/data_type_fixedlengthobject_serde.h @@ -99,6 +99,12 @@ class DataTypeFixedLengthObjectSerDe : public DataTypeSerDe { int row_idx, bool col_const) const override { return Status::NotSupported("write_column_to_pb with type " + column.get_name()); } + + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override { + return Status::NotSupported("write_column_to_orc with type [{}]", column.get_name()); + } }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_hll_serde.cpp b/be/src/vec/data_types/serde/data_type_hll_serde.cpp index e8cd09195fba51..d7057e77d9f9b4 100644 --- a/be/src/vec/data_types/serde/data_type_hll_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_hll_serde.cpp @@ -171,5 +171,23 @@ Status DataTypeHLLSerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeHLLSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, std::vector& buffer_list) const { + auto& col_data = assert_cast(column); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + const auto& ele = col_data.get_data_at(row_id); + cur_batch->data[row_id] = const_cast(ele.data); + cur_batch->length[row_id] = ele.size; + } + } + + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris \ No newline at end of file diff --git a/be/src/vec/data_types/serde/data_type_hll_serde.h b/be/src/vec/data_types/serde/data_type_hll_serde.h index 3612e85b612637..ec2b271ff4618d 100644 --- a/be/src/vec/data_types/serde/data_type_hll_serde.h +++ b/be/src/vec/data_types/serde/data_type_hll_serde.h @@ -65,6 +65,10 @@ class DataTypeHLLSerDe : public DataTypeSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + private: // Hll is binary data which is not shown by mysql. template diff --git a/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp b/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp index 1194d549af12de..678b071e0ad59b 100644 --- a/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp @@ -113,5 +113,11 @@ void DataTypeJsonbSerDe::write_column_to_arrow(const IColumn& column, const Null } } +Status DataTypeJsonbSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, std::vector& buffer_list) const { + return Status::NotSupported("write_column_to_orc with type [{}]", column.get_name()); +} + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_jsonb_serde.h b/be/src/vec/data_types/serde/data_type_jsonb_serde.h index a3c716e6d89e6a..8bfa8b7b6c35ae 100644 --- a/be/src/vec/data_types/serde/data_type_jsonb_serde.h +++ b/be/src/vec/data_types/serde/data_type_jsonb_serde.h @@ -55,6 +55,10 @@ class DataTypeJsonbSerDe : public DataTypeStringSerDe { int* num_deserialized, const FormatOptions& options, int nesting_level = 1) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_map_serde.cpp b/be/src/vec/data_types/serde/data_type_map_serde.cpp index 935c810dc86f70..f46962f59776ae 100644 --- a/be/src/vec/data_types/serde/data_type_map_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_map_serde.cpp @@ -468,5 +468,40 @@ Status DataTypeMapSerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeMapSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, std::vector& buffer_list) const { + orc::MapVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + cur_batch->offsets[0] = 0; + + auto& map_column = assert_cast(column); + const ColumnArray::Offsets64& offsets = map_column.get_offsets(); + const IColumn& nested_keys_column = map_column.get_keys(); + const IColumn& nested_values_column = map_column.get_values(); + + cur_batch->keys->resize(nested_keys_column.size()); + cur_batch->elements->resize(nested_values_column.size()); + + for (size_t row_id = start; row_id < end; row_id++) { + size_t offset = offsets[row_id - 1]; + size_t next_offset = offsets[row_id]; + + if (cur_batch->notNull[row_id] == 1) { + key_serde->write_column_to_orc(nested_keys_column, nullptr, cur_batch->keys.get(), + offset, next_offset, buffer_list); + value_serde->write_column_to_orc(nested_values_column, nullptr, + cur_batch->elements.get(), offset, next_offset, + buffer_list); + } + + cur_batch->offsets[row_id + 1] = next_offset; + } + cur_batch->keys->numElements = nested_keys_column.size(); + cur_batch->elements->numElements = nested_values_column.size(); + + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_map_serde.h b/be/src/vec/data_types/serde/data_type_map_serde.h index 54c269a0e5624e..e1cdc896c45e92 100644 --- a/be/src/vec/data_types/serde/data_type_map_serde.h +++ b/be/src/vec/data_types/serde/data_type_map_serde.h @@ -86,6 +86,10 @@ class DataTypeMapSerDe : public DataTypeSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + void set_return_object_as_string(bool value) override { DataTypeSerDe::set_return_object_as_string(value); key_serde->set_return_object_as_string(value); diff --git a/be/src/vec/data_types/serde/data_type_nullable_serde.cpp b/be/src/vec/data_types/serde/data_type_nullable_serde.cpp index 8e3f65b3a09fdb..cc3a97d154b0c2 100644 --- a/be/src/vec/data_types/serde/data_type_nullable_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_nullable_serde.cpp @@ -306,5 +306,26 @@ Status DataTypeNullableSerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeNullableSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, + std::vector& buffer_list) const { + const auto& column_nullable = assert_cast(column); + orc_col_batch->hasNulls = true; + + auto& null_map_tmp = column_nullable.get_null_map_data(); + auto orc_null_map = revert_null_map(&null_map_tmp, start, end); + // orc_col_batch->notNull.data() must add 'start' (+ start), + // because orc_col_batch->notNull.data() begins at 0 + // orc_null_map.data() do not need add 'start' (+ start), + // because orc_null_map begins at start and only has (end - start) elements + memcpy(orc_col_batch->notNull.data() + start, orc_null_map.data(), end - start); + + nested_serde->write_column_to_orc(column_nullable.get_nested_column(), + &column_nullable.get_null_map_data(), orc_col_batch, start, + end, buffer_list); + return Status::OK(); +} + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_nullable_serde.h b/be/src/vec/data_types/serde/data_type_nullable_serde.h index 9c5454cd834229..cde3c9db35157c 100644 --- a/be/src/vec/data_types/serde/data_type_nullable_serde.h +++ b/be/src/vec/data_types/serde/data_type_nullable_serde.h @@ -79,6 +79,10 @@ class DataTypeNullableSerDe : public DataTypeSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + void set_return_object_as_string(bool value) override { DataTypeSerDe::set_return_object_as_string(value); nested_serde->set_return_object_as_string(value); diff --git a/be/src/vec/data_types/serde/data_type_number_serde.cpp b/be/src/vec/data_types/serde/data_type_number_serde.cpp index cd650c72492bd8..4bbeb248d18e2a 100644 --- a/be/src/vec/data_types/serde/data_type_number_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_number_serde.cpp @@ -239,6 +239,72 @@ Status DataTypeNumberSerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +template +Status DataTypeNumberSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, + std::vector& buffer_list) const { + auto& col_data = assert_cast(column).get_data(); + + if constexpr (std::is_same_v) { + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); + if (!ptr) { + return Status::InternalError( + "malloc memory error when write largeint column data to orc file."); + } + StringRef bufferRef; + bufferRef.data = ptr; + bufferRef.size = BUFFER_UNIT_SIZE; + size_t offset = 0; + const size_t begin_off = offset; + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 0) { + continue; + } + std::string value_str = fmt::format("{}", col_data[row_id]); + size_t len = value_str.size(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + strcpy(const_cast(bufferRef.data) + offset, value_str.c_str()); + offset += len; + cur_batch->length[row_id] = len; + } + size_t data_off = 0; + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; + data_off += cur_batch->length[row_id]; + } + } + buffer_list.emplace_back(bufferRef); + cur_batch->numElements = end - start; + } else if constexpr ((std::is_integral::value && std::is_signed::value) || + std::is_same_v) { // tinyint/smallint/..int and boolean type + orc::LongVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + cur_batch->data[row_id] = col_data[row_id]; + } + } + cur_batch->numElements = end - start; + } else if constexpr (IsFloatNumber) { + orc::DoubleVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + cur_batch->data[row_id] = col_data[row_id]; + } + } + cur_batch->numElements = end - start; + } + return Status::OK(); +} + /// Explicit template instantiations - to avoid code bloat in headers. template class DataTypeNumberSerDe; template class DataTypeNumberSerDe; diff --git a/be/src/vec/data_types/serde/data_type_number_serde.h b/be/src/vec/data_types/serde/data_type_number_serde.h index 6c9bc5ab4ae3b8..b37c1578a83978 100644 --- a/be/src/vec/data_types/serde/data_type_number_serde.h +++ b/be/src/vec/data_types/serde/data_type_number_serde.h @@ -87,6 +87,10 @@ class DataTypeNumberSerDe : public DataTypeSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_object_serde.cpp b/be/src/vec/data_types/serde/data_type_object_serde.cpp index 273abae7a3e275..62a900175fee4f 100644 --- a/be/src/vec/data_types/serde/data_type_object_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_object_serde.cpp @@ -16,7 +16,28 @@ // under the License. #include "data_type_object_serde.h" + +#include "vec/columns/column_complex.h" namespace doris { -namespace vectorized {} +namespace vectorized { +Status DataTypeObjectSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, + std::vector& buffer_list) const { + auto& col_data = assert_cast(column); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + const auto& ele = col_data.get_data_at(row_id); + cur_batch->data[row_id] = const_cast(ele.data); + cur_batch->length[row_id] = ele.size; + } + } + + cur_batch->numElements = end - start; + return Status::OK(); +} +} // namespace vectorized } // namespace doris \ No newline at end of file diff --git a/be/src/vec/data_types/serde/data_type_object_serde.h b/be/src/vec/data_types/serde/data_type_object_serde.h index 2274e27c0358c4..cfa8343624d5b6 100644 --- a/be/src/vec/data_types/serde/data_type_object_serde.h +++ b/be/src/vec/data_types/serde/data_type_object_serde.h @@ -99,6 +99,10 @@ class DataTypeObjectSerDe : public DataTypeSerDe { int row_idx, bool col_const) const override { return Status::NotSupported("write_column_to_mysql with type " + column.get_name()); } + + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_quantilestate_serde.h b/be/src/vec/data_types/serde/data_type_quantilestate_serde.h index 081dff9315b382..bcb099f6963b40 100644 --- a/be/src/vec/data_types/serde/data_type_quantilestate_serde.h +++ b/be/src/vec/data_types/serde/data_type_quantilestate_serde.h @@ -121,6 +121,12 @@ class DataTypeQuantileStateSerDe : public DataTypeSerDe { return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override { + return Status::NotSupported("write_column_to_orc with type [{}]", column.get_name()); + } + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_serde.h b/be/src/vec/data_types/serde/data_type_serde.h index 057eac975673f6..7125ad589d44e5 100644 --- a/be/src/vec/data_types/serde/data_type_serde.h +++ b/be/src/vec/data_types/serde/data_type_serde.h @@ -20,6 +20,7 @@ #include #include +#include #include #include "arrow/status.h" @@ -40,6 +41,9 @@ class Array; namespace cctz { class time_zone; } // namespace cctz +namespace orc { +struct ColumnVectorBatch; +} // namespace orc #define SERIALIZE_COLUMN_TO_JSON() \ for (size_t i = start_idx; i < end_idx; ++i) { \ @@ -68,6 +72,19 @@ class time_zone; ++*num_deserialized; \ } +#define REALLOC_MEMORY_FOR_ORC_WRITER() \ + while (bufferRef.size - BUFFER_RESERVED_SIZE < offset + len) { \ + char* new_ptr = (char*)malloc(bufferRef.size + BUFFER_UNIT_SIZE); \ + if (!new_ptr) { \ + return Status::InternalError( \ + "malloc memory error when write largeint column data to orc file."); \ + } \ + memcpy(new_ptr, bufferRef.data, bufferRef.size); \ + free(const_cast(bufferRef.data)); \ + bufferRef.data = new_ptr; \ + bufferRef.size = bufferRef.size + BUFFER_UNIT_SIZE; \ + } + namespace doris { class PValues; class JsonbValue; @@ -156,6 +173,16 @@ class DataTypeSerDe { } }; + // only used for orc file format. + // Buffer used by date/datetime/datev2/datetimev2/largeint type + // date/datetime/datev2/datetimev2/largeint type will be converted to string bytes to store in Buffer + // The minimum value of largeint has 40 bytes after being converted to string(a negative number occupies a byte) + // The bytes of date/datetime/datev2/datetimev2 after converted to string are smaller than largeint + // Because a block is 4064 rows by default, here is 4064*40 bytes to BUFFER, + static constexpr size_t BUFFER_UNIT_SIZE = 4064 * 40; + // buffer reserves 40 bytes. The reserved space is just to prevent Headp-Buffer-Overflow + static constexpr size_t BUFFER_RESERVED_SIZE = 40; + public: DataTypeSerDe(); virtual ~DataTypeSerDe(); @@ -215,8 +242,6 @@ class DataTypeSerDe { int row_idx, bool col_const) const = 0; // Thrift serializer and deserializer - // ORC serializer and deserializer - // CSV serializer and deserializer // JSON serializer and deserializer @@ -228,6 +253,12 @@ class DataTypeSerDe { virtual void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const = 0; + // ORC serializer + virtual Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const = 0; + // ORC deserializer + virtual void set_return_object_as_string(bool value) { _return_object_as_string = value; } protected: diff --git a/be/src/vec/data_types/serde/data_type_string_serde.cpp b/be/src/vec/data_types/serde/data_type_string_serde.cpp index 22bf5cabe5fe06..77616347d9cb69 100644 --- a/be/src/vec/data_types/serde/data_type_string_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_string_serde.cpp @@ -231,5 +231,22 @@ Status DataTypeStringSerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeStringSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, + std::vector& buffer_list) const { + auto& col_data = assert_cast(column); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + for (size_t row_id = start; row_id < end; row_id++) { + const auto& ele = col_data.get_data_at(row_id); + cur_batch->data[row_id] = const_cast(ele.data); + cur_batch->length[row_id] = ele.size; + } + + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris \ No newline at end of file diff --git a/be/src/vec/data_types/serde/data_type_string_serde.h b/be/src/vec/data_types/serde/data_type_string_serde.h index 6d0a8a03412ce0..eb4c5627a02475 100644 --- a/be/src/vec/data_types/serde/data_type_string_serde.h +++ b/be/src/vec/data_types/serde/data_type_string_serde.h @@ -66,6 +66,10 @@ class DataTypeStringSerDe : public DataTypeSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_struct_serde.cpp b/be/src/vec/data_types/serde/data_type_struct_serde.cpp index 904cdc824b42b2..5803d536a23aba 100644 --- a/be/src/vec/data_types/serde/data_type_struct_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_struct_serde.cpp @@ -341,5 +341,33 @@ Status DataTypeStructSerDe::write_column_to_mysql(const IColumn& column, return _write_column_to_mysql(column, row_buffer, row_idx, col_const); } +Status DataTypeStructSerDe::write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, + std::vector& buffer_list) const { + orc::StructVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + const ColumnStruct& struct_col = assert_cast(column); + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + for (int i = 0; i < struct_col.tuple_size(); ++i) { + elemSerDeSPtrs[i]->write_column_to_orc(struct_col.get_column(i), nullptr, + cur_batch->fields[i], row_id, row_id + 1, + buffer_list); + } + } else { + // This else is necessary + // because we must set notNull when cur_batch->notNull[row_id] == 0 + for (int j = 0; j < struct_col.tuple_size(); ++j) { + cur_batch->fields[j]->hasNulls = true; + cur_batch->fields[j]->notNull[row_id] = 0; + } + } + } + + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris diff --git a/be/src/vec/data_types/serde/data_type_struct_serde.h b/be/src/vec/data_types/serde/data_type_struct_serde.h index 1f39375a1a3fd0..af10e2e87ab4a1 100644 --- a/be/src/vec/data_types/serde/data_type_struct_serde.h +++ b/be/src/vec/data_types/serde/data_type_struct_serde.h @@ -162,6 +162,10 @@ class DataTypeStructSerDe : public DataTypeSerDe { Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override; + Status write_column_to_orc(const IColumn& column, const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, int end, + std::vector& buffer_list) const override; + void set_return_object_as_string(bool value) override { DataTypeSerDe::set_return_object_as_string(value); for (auto& serde : elemSerDeSPtrs) { diff --git a/be/src/vec/exec/format/wal/wal_reader.cpp b/be/src/vec/exec/format/wal/wal_reader.cpp new file mode 100644 index 00000000000000..f616740e685e11 --- /dev/null +++ b/be/src/vec/exec/format/wal/wal_reader.cpp @@ -0,0 +1,77 @@ +// 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 "wal_reader.h" + +#include "common/logging.h" +#include "olap/wal_manager.h" +#include "runtime/runtime_state.h" +namespace doris::vectorized { +WalReader::WalReader(RuntimeState* state) : _state(state) { + _wal_id = state->wal_id(); +} +WalReader::~WalReader() { + if (_wal_reader.get() != nullptr) { + _wal_reader->finalize(); + } +} +Status WalReader::init_reader() { + RETURN_IF_ERROR(_state->exec_env()->wal_mgr()->get_wal_path(_wal_id, _wal_path)); + RETURN_IF_ERROR(_state->exec_env()->wal_mgr()->create_wal_reader(_wal_path, _wal_reader)); + return Status::OK(); +} +Status WalReader::get_next_block(Block* block, size_t* read_rows, bool* eof) { + PBlock pblock; + auto st = _wal_reader->read_block(pblock); + if (st.is()) { + LOG(INFO) << "read eof on wal:" << _wal_path; + *read_rows = 0; + *eof = true; + return Status::OK(); + } + if (!st.ok()) { + LOG(WARNING) << "Failed to read wal on path = " << _wal_path; + return st; + } + vectorized::Block tmp_block; + tmp_block.deserialize(pblock); + block->swap(tmp_block); + *read_rows = block->rows(); + VLOG_DEBUG << "read block rows:" << *read_rows; + return Status::OK(); +} + +void WalReader::string_split(const std::string& str, const std::string& splits, + std::vector& res) { + if (str == "") return; + std::string strs = str + splits; + size_t pos = strs.find(splits); + int step = splits.size(); + while (pos != strs.npos) { + std::string temp = strs.substr(0, pos); + res.push_back(temp); + strs = strs.substr(pos + step, strs.size()); + pos = strs.find(splits); + } +} + +Status WalReader::get_columns(std::unordered_map* name_to_type, + std::unordered_set* missing_cols) { + return Status::OK(); +} + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/exec/format/wal/wal_reader.h b/be/src/vec/exec/format/wal/wal_reader.h new file mode 100644 index 00000000000000..c204cd9b5e5a4e --- /dev/null +++ b/be/src/vec/exec/format/wal/wal_reader.h @@ -0,0 +1,43 @@ +// 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 "olap/wal_reader.h" +#include "vec/exec/format/generic_reader.h" +namespace doris { +namespace vectorized { +struct ScannerCounter; +class WalReader : public GenericReader { +public: + WalReader(RuntimeState* state); + ~WalReader() override; + Status init_reader(); + Status get_next_block(Block* block, size_t* read_rows, bool* eof) override; + Status get_columns(std::unordered_map* name_to_type, + std::unordered_set* missing_cols) override; + static void string_split(const std::string& str, const std::string& splits, + std::vector& res); + +private: + RuntimeState* _state; + std::string _wal_path; + std::string _path_split = "/"; + int64_t _wal_id; + std::shared_ptr _wal_reader = nullptr; +}; +} // namespace vectorized +} // namespace doris diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index 505b6807b439ca..dd96ef88a464e7 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -32,6 +32,7 @@ #include #include "vec/data_types/data_type_factory.hpp" +#include "vec/exec/format/wal/wal_reader.h" // IWYU pragma: no_include #include "common/compiler_util.h" // IWYU pragma: keep @@ -258,6 +259,10 @@ Status VFileScanner::_get_block_impl(RuntimeState* state, Block* block, bool* eo RETURN_IF_ERROR( _cur_reader->get_next_block(_src_block_ptr, &read_rows, &_cur_reader_eof)); } + if (_params->format_type == TFileFormatType::FORMAT_WAL) { + block->swap(*_src_block_ptr); + break; + } // use read_rows instead of _src_block_ptr->rows(), because the first column of _src_block_ptr // may not be filled after calling `get_next_block()`, so _src_block_ptr->rows() may return wrong result. if (read_rows > 0) { @@ -784,6 +789,11 @@ Status VFileScanner::_get_next_reader() { ->init_fetch_table_reader(_colname_to_value_range); break; } + case TFileFormatType::FORMAT_WAL: { + _cur_reader.reset(new WalReader(_state)); + init_status = ((WalReader*)(_cur_reader.get()))->init_reader(); + break; + } default: return Status::InternalError("Not supported file format: {}", _params->format_type); } diff --git a/be/src/vec/functions/function_tokenize.cpp b/be/src/vec/functions/function_tokenize.cpp index 62e0a53bccb268..11760a30f5025f 100644 --- a/be/src/vec/functions/function_tokenize.cpp +++ b/be/src/vec/functions/function_tokenize.cpp @@ -20,6 +20,7 @@ #include #include +#include #include #include "CLucene/StdHeader.h" @@ -29,46 +30,26 @@ #include "vec/common/string_ref.h" #include "vec/core/block.h" #include "vec/core/column_with_type_and_name.h" -#include "vec/core/field.h" #include "vec/data_types/data_type_nullable.h" #include "vec/data_types/data_type_number.h" -#include "vec/functions/simple_function_factory.h" namespace doris::vectorized { Status parse(const std::string& str, std::map& result) { - std::string::size_type start = 0; + std::regex pattern( + R"delimiter((?:'([^']*)'|"([^"]*)"|([^,]*))\s*=\s*(?:'([^']*)'|"([^"]*)"|([^,]*)))delimiter"); + std::smatch matches; - while (start < str.size()) { - std::string::size_type end = str.find(',', start); - std::string pair = - (end == std::string::npos) ? str.substr(start) : str.substr(start, end - start); - - std::string::size_type eq_pos = pair.find('='); - if (eq_pos == std::string::npos) { - return Status::InvalidArgument( - fmt::format("invalid params {} for function tokenize", str)); - } - std::string key = pair.substr(0, eq_pos); - key = key.substr(key.find_first_not_of(" '\"" - "\t\n\r"), - key.find_last_not_of(" '\"" - "\t\n\r") - - key.find_first_not_of(" '\"" - "\t\n\r") + - 1); - std::string value = pair.substr(eq_pos + 1); - value = value.substr(value.find_first_not_of(" '\"" - "\t\n\r"), - value.find_last_not_of(" '\"" - "\t\n\r") - - value.find_first_not_of(" '\"" - "\t\n\r") + - 1); + std::string::const_iterator searchStart(str.cbegin()); + while (std::regex_search(searchStart, str.cend(), matches, pattern)) { + std::string key = + matches[1].length() ? matches[1] : (matches[2].length() ? matches[2] : matches[3]); + std::string value = + matches[4].length() ? matches[4] : (matches[5].length() ? matches[5] : matches[6]); result[key] = value; - start = (end == std::string::npos) ? str.size() : end + 1; + searchStart = matches.suffix().first; } return Status::OK(); @@ -169,6 +150,6 @@ Status FunctionTokenize::execute_impl(FunctionContext* /*context*/, Block& block return Status::OK(); } } - return Status::RuntimeError("unimplements function {}", get_name()); + return Status::RuntimeError("unimplemented function {}", get_name()); } } // namespace doris::vectorized diff --git a/be/src/vec/runtime/vfile_format_transformer.h b/be/src/vec/runtime/vfile_format_transformer.h index e94f6b0ead6459..2b5440fdfd57fa 100644 --- a/be/src/vec/runtime/vfile_format_transformer.h +++ b/be/src/vec/runtime/vfile_format_transformer.h @@ -22,6 +22,9 @@ #include "common/status.h" #include "vec/core/block.h" +#include "vec/data_types/data_type.h" +#include "vec/exprs/vexpr.h" +#include "vec/exprs/vexpr_context.h" #include "vec/exprs/vexpr_fwd.h" namespace doris::vectorized { @@ -31,21 +34,26 @@ class VFileFormatTransformer { VFileFormatTransformer(const VExprContextSPtrs& output_vexpr_ctxs, bool output_object_data) : _output_vexpr_ctxs(output_vexpr_ctxs), _cur_written_rows(0), - _output_object_data(output_object_data) {} + _output_object_data(output_object_data) { + DataTypes data_types; + for (int i = 0; i < output_vexpr_ctxs.size(); ++i) { + data_types.push_back(output_vexpr_ctxs[i]->root()->data_type()); + } + _serdes = vectorized::create_data_type_serdes(data_types); + } virtual ~VFileFormatTransformer() = default; virtual Status open() = 0; - virtual Status write(const Block& block) = 0; - virtual Status close() = 0; - virtual int64_t written_len() = 0; protected: const VExprContextSPtrs& _output_vexpr_ctxs; int64_t _cur_written_rows; bool _output_object_data; + vectorized::DataTypeSerDeSPtrs _serdes; + vectorized::DataTypeSerDe::FormatOptions _options; }; } // namespace doris::vectorized diff --git a/be/src/vec/runtime/vorc_transformer.cpp b/be/src/vec/runtime/vorc_transformer.cpp index 2acede5d4be29c..e862624c905453 100644 --- a/be/src/vec/runtime/vorc_transformer.cpp +++ b/be/src/vec/runtime/vorc_transformer.cpp @@ -130,277 +130,6 @@ Status VOrcTransformer::close() { return Status::OK(); } -#define RETURN_WRONG_TYPE \ - return Status::InvalidArgument("Invalid column type: {}", raw_column->get_name()); - -#define WRITE_SINGLE_ELEMENTS_INTO_BATCH(VECTOR_BATCH, COLUMN) \ - VECTOR_BATCH* cur_batch = dynamic_cast(orc_col_batch); \ - if (null_map != nullptr) { \ - cur_batch->hasNulls = true; \ - auto& null_data = assert_cast(*null_map).get_data(); \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->notNull[row_id] = 1; \ - cur_batch->data[row_id] = assert_cast(*col).get_data()[row_id]; \ - } \ - } \ - } else if (const auto& not_null_column = check_and_get_column(col)) { \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - cur_batch->data[row_id] = not_null_column->get_data()[row_id]; \ - } \ - } else { \ - RETURN_WRONG_TYPE \ - } - -#define WRITE_CONTINUOUS_ELEMENTS_INTO_BATCH(VECTOR_BATCH, COLUMN, NATIVE_TYPE) \ - VECTOR_BATCH* cur_batch = dynamic_cast(orc_col_batch); \ - if (null_map != nullptr) { \ - cur_batch->hasNulls = true; \ - auto& null_data = assert_cast(*null_map).get_data(); \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->notNull[row_id] = 1; \ - cur_batch->data[row_id] = assert_cast(*col).get_data()[row_id]; \ - } \ - } \ - } else if (const auto& not_null_column = check_and_get_column(col)) { \ - memcpy(cur_batch->data.data(), not_null_column->get_data().data(), \ - (end_row_id - start_row_id) * sizeof(NATIVE_TYPE)); \ - } else { \ - RETURN_WRONG_TYPE \ - } - -#define WRITE_LARGEINT_STRING_INTO_BATCH(VECTOR_BATCH, COLUMN, BUFFER) \ - VECTOR_BATCH* cur_batch = dynamic_cast(orc_col_batch); \ - const size_t begin_off = offset; \ - if (null_map != nullptr) { \ - cur_batch->hasNulls = true; \ - auto& null_data = assert_cast(*null_map).get_data(); \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->notNull[row_id] = 1; \ - auto value = assert_cast(*col).get_data()[row_id]; \ - std::string value_str = fmt::format("{}", value); \ - size_t len = value_str.size(); \ - while (BUFFER.size - BUFFER_RESERVED_SIZE < offset + len) { \ - char* new_ptr = (char*)malloc(BUFFER.size + BUFFER_UNIT_SIZE); \ - memcpy(new_ptr, BUFFER.data, BUFFER.size); \ - free(const_cast(BUFFER.data)); \ - BUFFER.data = new_ptr; \ - BUFFER.size = BUFFER.size + BUFFER_UNIT_SIZE; \ - } \ - strcpy(const_cast(BUFFER.data) + offset, value_str.c_str()); \ - offset += len; \ - cur_batch->length[row_id] = len; \ - } \ - } \ - size_t data_off = 0; \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ - data_off += cur_batch->length[row_id]; \ - } \ - } \ - } else if (const auto& not_null_column = check_and_get_column(col)) { \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - auto value = not_null_column->get_data()[row_id]; \ - std::string value_str = fmt::format("{}", value); \ - size_t len = value_str.size(); \ - while (BUFFER.size - BUFFER_RESERVED_SIZE < offset + len) { \ - char* new_ptr = (char*)malloc(BUFFER.size + BUFFER_UNIT_SIZE); \ - memcpy(new_ptr, BUFFER.data, BUFFER.size); \ - free(const_cast(BUFFER.data)); \ - BUFFER.data = new_ptr; \ - BUFFER.size = BUFFER.size + BUFFER_UNIT_SIZE; \ - } \ - strcpy(const_cast(BUFFER.data) + offset, value_str.c_str()); \ - offset += len; \ - cur_batch->length[row_id] = len; \ - } \ - size_t data_off = 0; \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ - data_off += cur_batch->length[row_id]; \ - } \ - } else { \ - RETURN_WRONG_TYPE \ - } - -#define WRITE_DATE_STRING_INTO_BATCH(FROM, TO, BUFFER) \ - orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); \ - const size_t begin_off = offset; \ - if (null_map != nullptr) { \ - cur_batch->hasNulls = true; \ - auto& null_data = assert_cast(*null_map).get_data(); \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->notNull[row_id] = 1; \ - int len = binary_cast( \ - assert_cast&>(*col).get_data()[row_id]) \ - .to_buffer(const_cast(BUFFER.data) + offset); \ - while (BUFFER.size - BUFFER_RESERVED_SIZE < offset + len) { \ - char* new_ptr = (char*)malloc(BUFFER.size + BUFFER_UNIT_SIZE); \ - memcpy(new_ptr, BUFFER.data, BUFFER.size); \ - free(const_cast(BUFFER.data)); \ - BUFFER.data = new_ptr; \ - BUFFER.size = BUFFER.size + BUFFER_UNIT_SIZE; \ - } \ - cur_batch->length[row_id] = len; \ - offset += len; \ - } \ - } \ - size_t data_off = 0; \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ - data_off += cur_batch->length[row_id]; \ - } \ - } \ - } else if (const auto& not_null_column = \ - check_and_get_column>(col)) { \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - int len = binary_cast(not_null_column->get_data()[row_id]) \ - .to_buffer(const_cast(BUFFER.data) + offset); \ - while (BUFFER.size - BUFFER_RESERVED_SIZE < offset + len) { \ - char* new_ptr = (char*)malloc(BUFFER.size + BUFFER_UNIT_SIZE); \ - memcpy(new_ptr, BUFFER.data, BUFFER.size); \ - free(const_cast(BUFFER.data)); \ - BUFFER.data = new_ptr; \ - BUFFER.size = BUFFER.size + BUFFER_UNIT_SIZE; \ - } \ - cur_batch->length[row_id] = len; \ - offset += len; \ - } \ - size_t data_off = 0; \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ - data_off += cur_batch->length[row_id]; \ - } \ - } else { \ - RETURN_WRONG_TYPE \ - } - -#define WRITE_DATETIMEV2_STRING_INTO_BATCH(FROM, TO, BUFFER) \ - orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); \ - const size_t begin_off = offset; \ - if (null_map != nullptr) { \ - cur_batch->hasNulls = true; \ - auto& null_data = assert_cast(*null_map).get_data(); \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->notNull[row_id] = 1; \ - int output_scale = type_descriptor.scale; \ - int len = \ - binary_cast( \ - assert_cast&>(*col).get_data()[row_id]) \ - .to_buffer(const_cast(BUFFER.data) + offset, output_scale); \ - while (BUFFER.size - BUFFER_RESERVED_SIZE < offset + len) { \ - char* new_ptr = (char*)malloc(BUFFER.size + BUFFER_UNIT_SIZE); \ - memcpy(new_ptr, BUFFER.data, BUFFER.size); \ - free(const_cast(BUFFER.data)); \ - BUFFER.data = new_ptr; \ - BUFFER.size = BUFFER.size + BUFFER_UNIT_SIZE; \ - } \ - cur_batch->length[row_id] = len; \ - offset += len; \ - } \ - } \ - size_t data_off = 0; \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ - data_off += cur_batch->length[row_id]; \ - } \ - } \ - } else if (const auto& not_null_column = \ - check_and_get_column>(col)) { \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - int output_scale = type_descriptor.scale; \ - int len = binary_cast(not_null_column->get_data()[row_id]) \ - .to_buffer(const_cast(BUFFER.data) + offset, output_scale); \ - while (BUFFER.size - BUFFER_RESERVED_SIZE < offset + len) { \ - char* new_ptr = (char*)malloc(BUFFER.size + BUFFER_UNIT_SIZE); \ - memcpy(new_ptr, BUFFER.data, BUFFER.size); \ - free(const_cast(BUFFER.data)); \ - BUFFER.data = new_ptr; \ - BUFFER.size = BUFFER.size + BUFFER_UNIT_SIZE; \ - } \ - cur_batch->length[row_id] = len; \ - offset += len; \ - } \ - size_t data_off = 0; \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ - data_off += cur_batch->length[row_id]; \ - } \ - } else { \ - RETURN_WRONG_TYPE \ - } - -#define WRITE_DECIMAL_INTO_BATCH(VECTOR_BATCH, COLUMN) \ - VECTOR_BATCH* cur_batch = dynamic_cast(orc_col_batch); \ - if (null_map != nullptr) { \ - cur_batch->hasNulls = true; \ - auto& null_data = assert_cast(*null_map).get_data(); \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->notNull[row_id] = 1; \ - cur_batch->values[row_id] = assert_cast(*col).get_data()[row_id]; \ - } \ - } \ - } else if (const auto& not_null_column = check_and_get_column(col)) { \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - cur_batch->values[row_id] = not_null_column->get_data()[row_id]; \ - } \ - } else { \ - RETURN_WRONG_TYPE \ - } - -#define WRITE_COMPLEX_TYPE_INTO_BATCH(VECTOR_BATCH, COLUMN) \ - VECTOR_BATCH* cur_batch = dynamic_cast(orc_col_batch); \ - if (null_map != nullptr) { \ - cur_batch->hasNulls = true; \ - auto& null_data = assert_cast(*null_map).get_data(); \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - if (null_data[row_id] != 0) { \ - cur_batch->notNull[row_id] = 0; \ - } else { \ - cur_batch->notNull[row_id] = 1; \ - const auto& ele = col->get_data_at(row_id); \ - cur_batch->data[row_id] = const_cast(ele.data); \ - cur_batch->length[row_id] = ele.size; \ - } \ - } \ - } else if (const auto& not_null_column = check_and_get_column(col)) { \ - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { \ - const auto& ele = not_null_column->get_data_at(row_id); \ - cur_batch->data[row_id] = const_cast(ele.data); \ - cur_batch->length[row_id] = ele.size; \ - } \ - } else { \ - RETURN_WRONG_TYPE \ - } - -#define SET_NUM_ELEMENTS cur_batch->numElements = end_row_id - start_row_id; - Status VOrcTransformer::write(const Block& block) { if (block.rows() == 0) { return Status::OK(); @@ -422,8 +151,8 @@ Status VOrcTransformer::write(const Block& block) { try { for (size_t i = 0; i < block.columns(); i++) { auto& raw_column = block.get_by_position(i).column; - _write_one_col(_output_vexpr_ctxs[i]->root()->type(), root->fields[i], raw_column, 0, - sz, &buffer_list); + RETURN_IF_ERROR(_serdes[i]->write_column_to_orc(*raw_column, nullptr, root->fields[i], + 0, sz, buffer_list)); } } catch (const std::exception& e) { LOG(WARNING) << "Orc write error: " << e.what(); @@ -436,345 +165,4 @@ Status VOrcTransformer::write(const Block& block) { return Status::OK(); } -Status VOrcTransformer::_write_one_col(const TypeDescriptor& type_descriptor, - orc::ColumnVectorBatch* orc_col_batch, - const ColumnPtr& raw_column, size_t start_row_id, - size_t end_row_id, std::vector* buffer_list) { - auto nullable = raw_column->is_nullable(); - const auto col = nullable ? reinterpret_cast(raw_column.get()) - ->get_nested_column_ptr() - .get() - : raw_column.get(); - auto null_map = nullable ? reinterpret_cast(raw_column.get()) - ->get_null_map_column_ptr() - : nullptr; - switch (type_descriptor.type) { - case TYPE_BOOLEAN: { - WRITE_SINGLE_ELEMENTS_INTO_BATCH(orc::LongVectorBatch, ColumnVector) - SET_NUM_ELEMENTS - break; - } - case TYPE_TINYINT: { - WRITE_SINGLE_ELEMENTS_INTO_BATCH(orc::LongVectorBatch, ColumnVector) - SET_NUM_ELEMENTS - break; - } - case TYPE_SMALLINT: { - WRITE_SINGLE_ELEMENTS_INTO_BATCH(orc::LongVectorBatch, ColumnVector) - SET_NUM_ELEMENTS - break; - } - case TYPE_INT: { - WRITE_SINGLE_ELEMENTS_INTO_BATCH(orc::LongVectorBatch, ColumnVector) - SET_NUM_ELEMENTS - break; - } - case TYPE_BIGINT: { - WRITE_CONTINUOUS_ELEMENTS_INTO_BATCH(orc::LongVectorBatch, ColumnVector, Int64) - SET_NUM_ELEMENTS - break; - } - case TYPE_LARGEINT: { - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - WRITE_LARGEINT_STRING_INTO_BATCH(orc::StringVectorBatch, ColumnVector, bufferRef) - buffer_list->emplace_back(bufferRef); - SET_NUM_ELEMENTS; - break; - } - case TYPE_FLOAT: { - WRITE_SINGLE_ELEMENTS_INTO_BATCH(orc::DoubleVectorBatch, ColumnVector) - SET_NUM_ELEMENTS - break; - } - case TYPE_DOUBLE: { - WRITE_CONTINUOUS_ELEMENTS_INTO_BATCH(orc::DoubleVectorBatch, ColumnVector, Float64) - SET_NUM_ELEMENTS - break; - } - case TYPE_DATETIME: - case TYPE_DATE: { - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - WRITE_DATE_STRING_INTO_BATCH(Int64, VecDateTimeValue, bufferRef) - buffer_list->emplace_back(bufferRef); - SET_NUM_ELEMENTS - break; - } - case TYPE_DATEV2: { - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - WRITE_DATE_STRING_INTO_BATCH(UInt32, DateV2Value, bufferRef) - buffer_list->emplace_back(bufferRef); - SET_NUM_ELEMENTS - break; - } - case TYPE_DATETIMEV2: { - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - WRITE_DATETIMEV2_STRING_INTO_BATCH(UInt64, DateV2Value, bufferRef) - buffer_list->emplace_back(bufferRef); - SET_NUM_ELEMENTS - break; - } - case TYPE_OBJECT: { - if (_output_object_data) { - WRITE_COMPLEX_TYPE_INTO_BATCH(orc::StringVectorBatch, ColumnBitmap) - SET_NUM_ELEMENTS - } else { - RETURN_WRONG_TYPE - } - break; - } - case TYPE_HLL: { - if (_output_object_data) { - WRITE_COMPLEX_TYPE_INTO_BATCH(orc::StringVectorBatch, ColumnHLL) - SET_NUM_ELEMENTS - } else { - RETURN_WRONG_TYPE - } - break; - } - case TYPE_CHAR: - case TYPE_VARCHAR: - case TYPE_STRING: { - WRITE_COMPLEX_TYPE_INTO_BATCH(orc::StringVectorBatch, ColumnString) - SET_NUM_ELEMENTS - break; - } - case TYPE_DECIMAL32: { - WRITE_DECIMAL_INTO_BATCH(orc::Decimal64VectorBatch, ColumnDecimal32) - SET_NUM_ELEMENTS - break; - } - case TYPE_DECIMAL64: { - WRITE_DECIMAL_INTO_BATCH(orc::Decimal64VectorBatch, ColumnDecimal64) - SET_NUM_ELEMENTS - break; - } - case TYPE_DECIMALV2: { - orc::Decimal128VectorBatch* cur_batch = - dynamic_cast(orc_col_batch); - if (null_map != nullptr) { - cur_batch->hasNulls = true; - auto& null_data = assert_cast(*null_map).get_data(); - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - if (null_data[row_id] != 0) { - cur_batch->notNull[row_id] = 0; - } else { - cur_batch->notNull[row_id] = 1; - auto& v = assert_cast(*col).get_data()[row_id]; - orc::Int128 value(v >> 64, (uint64_t)v); - cur_batch->values[row_id] = value; - } - } - } else if (const auto& not_null_column = - check_and_get_column(col)) { - auto col_ptr = not_null_column->get_data().data(); - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - auto v = col_ptr[row_id]; - orc::Int128 value(v >> 64, (uint64_t)v); - cur_batch->values[row_id] = value; - } - } else { - RETURN_WRONG_TYPE - } - SET_NUM_ELEMENTS - break; - } - case TYPE_DECIMAL128I: { - orc::Decimal128VectorBatch* cur_batch = - dynamic_cast(orc_col_batch); - if (null_map != nullptr) { - cur_batch->hasNulls = true; - auto& null_data = assert_cast(*null_map).get_data(); - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - if (null_data[row_id] != 0) { - cur_batch->notNull[row_id] = 0; - } else { - cur_batch->notNull[row_id] = 1; - auto& v = assert_cast(*col).get_data()[row_id]; - orc::Int128 value(v.value >> 64, (uint64_t)v.value); - cur_batch->values[row_id] = value; - } - } - } else if (const auto& not_null_column = - check_and_get_column(col)) { - auto col_ptr = not_null_column->get_data().data(); - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - auto v = col_ptr[row_id].value; - orc::Int128 value(v >> 64, (uint64_t)v); - cur_batch->values[row_id] = value; - } - } else { - RETURN_WRONG_TYPE - } - SET_NUM_ELEMENTS - break; - } - case TYPE_STRUCT: { - orc::StructVectorBatch* cur_batch = dynamic_cast(orc_col_batch); - - if (null_map != nullptr) { - cur_batch->hasNulls = true; - auto& null_data = assert_cast(*null_map).get_data(); - const ColumnStruct& struct_col = assert_cast(*col); - - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - if (null_data[row_id] != 0) { - cur_batch->notNull[row_id] = 0; - for (int j = 0; j < struct_col.tuple_size(); ++j) { - cur_batch->fields[j]->hasNulls = true; - cur_batch->fields[j]->notNull[row_id] = 0; - } - } else { - cur_batch->notNull[row_id] = 1; - - for (int j = 0; j < struct_col.tuple_size(); ++j) { - _write_one_col(type_descriptor.children[j], cur_batch->fields[j], - struct_col.get_column_ptr(j), row_id, row_id + 1, - buffer_list); - } - } - } - - } else if (const auto& not_null_column = check_and_get_column(col)) { - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - for (int j = 0; j < not_null_column->tuple_size(); ++j) { - _write_one_col(type_descriptor.children[j], cur_batch->fields[j], - not_null_column->get_column_ptr(j), row_id, row_id + 1, - buffer_list); - } - } - } else { - RETURN_WRONG_TYPE - } - SET_NUM_ELEMENTS - break; - } - case TYPE_MAP: { - orc::MapVectorBatch* cur_batch = dynamic_cast(orc_col_batch); - cur_batch->offsets[0] = 0; - - if (null_map != nullptr) { - cur_batch->hasNulls = true; - auto& null_data = assert_cast(*null_map).get_data(); - - const ColumnMap& map_column = assert_cast(*col); - const ColumnArray::Offsets64& offsets = map_column.get_offsets(); - const ColumnPtr& nested_keys_column = map_column.get_keys_ptr(); - const ColumnPtr& nested_values_column = map_column.get_values_ptr(); - - cur_batch->keys->resize(nested_keys_column->size()); - cur_batch->elements->resize(nested_values_column->size()); - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - size_t offset = offsets[row_id - 1]; - size_t next_offset = offsets[row_id]; - if (null_data[row_id] != 0) { - cur_batch->notNull[row_id] = 0; - } else { - cur_batch->notNull[row_id] = 1; - _write_one_col(type_descriptor.children[0], cur_batch->keys.get(), - nested_keys_column, offset, next_offset, buffer_list); - _write_one_col(type_descriptor.children[1], cur_batch->elements.get(), - nested_values_column, offset, next_offset, buffer_list); - } - cur_batch->offsets[row_id + 1] = next_offset; - } - cur_batch->keys->numElements = nested_keys_column->size(); - cur_batch->elements->numElements = nested_values_column->size(); - - } else if (const auto& not_null_column = check_and_get_column(col)) { - const ColumnArray::Offsets64& offsets = not_null_column->get_offsets(); - const ColumnPtr& nested_keys_column = not_null_column->get_keys_ptr(); - const ColumnPtr& nested_values_column = not_null_column->get_values_ptr(); - - cur_batch->keys->resize(nested_keys_column->size()); - cur_batch->elements->resize(nested_values_column->size()); - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - size_t offset = offsets[row_id - 1]; - size_t next_offset = offsets[row_id]; - - _write_one_col(type_descriptor.children[0], cur_batch->keys.get(), - nested_keys_column, offset, next_offset, buffer_list); - _write_one_col(type_descriptor.children[1], cur_batch->elements.get(), - nested_values_column, offset, next_offset, buffer_list); - - cur_batch->offsets[row_id + 1] = next_offset; - } - cur_batch->keys->numElements = nested_keys_column->size(); - cur_batch->elements->numElements = nested_values_column->size(); - - } else { - RETURN_WRONG_TYPE - } - SET_NUM_ELEMENTS - break; - } - case TYPE_ARRAY: { - orc::ListVectorBatch* cur_batch = dynamic_cast(orc_col_batch); - cur_batch->offsets[0] = 0; - if (null_map != nullptr) { - cur_batch->hasNulls = true; - auto& null_data = assert_cast(*null_map).get_data(); - - const ColumnArray& array_col = assert_cast(*col); - const ColumnPtr& nested_column = array_col.get_data_ptr(); - - cur_batch->elements->resize(nested_column->size()); - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - auto& offsets = array_col.get_offsets(); - size_t offset = offsets[row_id - 1]; - size_t next_offset = offsets[row_id]; - if (null_data[row_id] != 0) { - cur_batch->notNull[row_id] = 0; - } else { - cur_batch->notNull[row_id] = 1; - _write_one_col(type_descriptor.children[0], cur_batch->elements.get(), - nested_column, offset, next_offset, buffer_list); - } - - cur_batch->offsets[row_id + 1] = next_offset; - } - cur_batch->elements->numElements = nested_column->size(); - - } else if (const auto& not_null_column = check_and_get_column(col)) { - auto& offsets = not_null_column->get_offsets(); - const ColumnPtr& nested_column = not_null_column->get_data_ptr(); - cur_batch->elements->resize(nested_column->size()); - for (size_t row_id = start_row_id; row_id < end_row_id; row_id++) { - size_t offset = offsets[row_id - 1]; - size_t next_offset = offsets[row_id]; - _write_one_col(type_descriptor.children[0], cur_batch->elements.get(), - nested_column, offset, next_offset, buffer_list); - - cur_batch->offsets[row_id + 1] = next_offset; - } - cur_batch->elements->numElements = nested_column->size(); - } else { - RETURN_WRONG_TYPE - } - SET_NUM_ELEMENTS - break; - } - default: { - return Status::InvalidArgument("Invalid expression type: {}", - type_descriptor.debug_string()); - } - } - return Status::OK(); -} - } // namespace doris::vectorized diff --git a/be/src/vec/runtime/vorc_transformer.h b/be/src/vec/runtime/vorc_transformer.h index 1a94901e22f2a8..06a42361fb53ff 100644 --- a/be/src/vec/runtime/vorc_transformer.h +++ b/be/src/vec/runtime/vorc_transformer.h @@ -89,9 +89,6 @@ class VOrcTransformer final : public VFileFormatTransformer { private: std::unique_ptr _create_row_batch(size_t sz); - Status _write_one_col(const TypeDescriptor& type, orc::ColumnVectorBatch* orc_col_batch, - const ColumnPtr& raw_column, size_t start_row_id, size_t end_row_id, - std::vector* buffer_list); doris::io::FileWriter* _file_writer; std::unique_ptr _output_stream; diff --git a/be/src/vec/sink/async_writer_sink.h b/be/src/vec/sink/async_writer_sink.h index 5fd03762a5580a..b0d772a841d582 100644 --- a/be/src/vec/sink/async_writer_sink.h +++ b/be/src/vec/sink/async_writer_sink.h @@ -90,7 +90,8 @@ class AsyncWriterSink : public DataSink { bool can_write() override { return _writer->can_write(); } Status close(RuntimeState* state, Status exec_status) override { - if (_writer->need_normal_close()) { + // if the init failed, the _writer may be nullptr. so here need check + if (_writer && _writer->need_normal_close()) { if (exec_status.ok() && !state->is_cancelled()) { RETURN_IF_ERROR(_writer->commit_trans()); } diff --git a/be/src/vec/sink/group_commit_block_sink.cpp b/be/src/vec/sink/group_commit_block_sink.cpp new file mode 100644 index 00000000000000..08f6d87ade732a --- /dev/null +++ b/be/src/vec/sink/group_commit_block_sink.cpp @@ -0,0 +1,105 @@ +// 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_block_sink.h" + +#include "runtime/group_commit_mgr.h" +#include "runtime/runtime_state.h" +#include "util/doris_metrics.h" +#include "vec/exprs/vexpr.h" +#include "vec/sink/vtablet_finder.h" +#include "vec/sink/vtablet_sink.h" + +namespace doris { + +namespace stream_load { + +GroupCommitBlockSink::GroupCommitBlockSink(ObjectPool* pool, const RowDescriptor& row_desc, + const std::vector& texprs, Status* status) + : DataSink(row_desc) { + // From the thrift expressions create the real exprs. + *status = vectorized::VExpr::create_expr_trees(texprs, _output_vexpr_ctxs); + _name = "GroupCommitBlockSink"; +} + +GroupCommitBlockSink::~GroupCommitBlockSink() = default; + +Status GroupCommitBlockSink::init(const TDataSink& t_sink) { + DCHECK(t_sink.__isset.olap_table_sink); + auto& table_sink = t_sink.olap_table_sink; + _tuple_desc_id = table_sink.tuple_id; + _schema.reset(new OlapTableSchemaParam()); + RETURN_IF_ERROR(_schema->init(table_sink.schema)); + return Status::OK(); +} + +Status GroupCommitBlockSink::prepare(RuntimeState* state) { + RETURN_IF_ERROR(DataSink::prepare(state)); + _state = state; + + // 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()); + // Prepare the exprs to run. + return vectorized::VExpr::prepare(_output_vexpr_ctxs, state, _row_desc); +} + +Status GroupCommitBlockSink::open(RuntimeState* state) { + // Prepare the exprs to run. + return vectorized::VExpr::open(_output_vexpr_ctxs, state); +} + +Status GroupCommitBlockSink::send(RuntimeState* state, vectorized::Block* input_block, bool eos) { + SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); + Status status = Status::OK(); + auto rows = input_block->rows(); + auto bytes = input_block->bytes(); + if (UNLIKELY(rows == 0)) { + return status; + } + SCOPED_TIMER(_profile->total_time_counter()); + // 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; + RETURN_IF_ERROR(_block_convertor->validate_and_convert_block( + state, input_block, block, _output_vexpr_ctxs, rows, has_filtered_rows)); + block->swap(*input_block); + return Status::OK(); +} + +} // namespace stream_load +} // namespace doris diff --git a/be/src/vec/sink/group_commit_block_sink.h b/be/src/vec/sink/group_commit_block_sink.h new file mode 100644 index 00000000000000..a309413f5adc5d --- /dev/null +++ b/be/src/vec/sink/group_commit_block_sink.h @@ -0,0 +1,59 @@ +// 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 "exec/data_sink.h" +#include "vec/exprs/vexpr_fwd.h" +#include "vec/sink/vtablet_sink.h" + +namespace doris { + +class OlapTableSchemaParam; +class MemTracker; + +namespace stream_load { + +class GroupCommitBlockSink : public DataSink { +public: + GroupCommitBlockSink(ObjectPool* pool, const RowDescriptor& row_desc, + const std::vector& texprs, Status* status); + + ~GroupCommitBlockSink() override; + + Status init(const TDataSink& sink) override; + + Status prepare(RuntimeState* state) override; + + Status open(RuntimeState* state) override; + + Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override; + +private: + vectorized::VExprContextSPtrs _output_vexpr_ctxs; + + int _tuple_desc_id = -1; + std::shared_ptr _schema; + + RuntimeState* _state = nullptr; + std::shared_ptr _mem_tracker; + // this is tuple descriptor of destination OLAP table + TupleDescriptor* _output_tuple_desc = nullptr; + std::unique_ptr _block_convertor; +}; + +} // namespace stream_load +} // namespace doris diff --git a/be/src/vec/sink/vtablet_finder.cpp b/be/src/vec/sink/vtablet_finder.cpp index 5d030a7ffab88a..421b3ebb11ccf1 100644 --- a/be/src/vec/sink/vtablet_finder.cpp +++ b/be/src/vec/sink/vtablet_finder.cpp @@ -66,6 +66,7 @@ Status OlapTabletFinder::find_tablet(RuntimeState* state, Block* block, int row_ }, &stop_processing)); _num_filtered_rows++; + _filter_bitmap.Set(row_index, true); if (stop_processing) { return Status::EndOfFile("Encountered unqualified data, stop processing"); } diff --git a/be/src/vec/sink/vtablet_finder.h b/be/src/vec/sink/vtablet_finder.h index 2258e15de08161..28d71c6a1e7e88 100644 --- a/be/src/vec/sink/vtablet_finder.h +++ b/be/src/vec/sink/vtablet_finder.h @@ -21,6 +21,7 @@ #include "common/status.h" #include "exec/tablet_info.h" +#include "util/bitmap.h" #include "vec/core/block.h" namespace doris::vectorized { @@ -36,7 +37,7 @@ class OlapTabletFinder { enum FindTabletMode { FIND_TABLET_EVERY_ROW, FIND_TABLET_EVERY_BATCH, FIND_TABLET_EVERY_SINK }; OlapTabletFinder(VOlapTablePartitionParam* vpartition, FindTabletMode mode) - : _vpartition(vpartition), _find_tablet_mode(mode) {}; + : _vpartition(vpartition), _find_tablet_mode(mode), _filter_bitmap(1024) {}; Status find_tablet(RuntimeState* state, vectorized::Block* block, int row_index, const VOlapTablePartition** partition, uint32_t& tablet_index, @@ -62,6 +63,8 @@ class OlapTabletFinder { return _num_immutable_partition_filtered_rows; } + Bitmap& filter_bitmap() { return _filter_bitmap; } + private: VOlapTablePartitionParam* _vpartition; FindTabletMode _find_tablet_mode; @@ -70,6 +73,7 @@ class OlapTabletFinder { int64_t _num_filtered_rows = 0; int64_t _num_immutable_partition_filtered_rows = 0; + Bitmap _filter_bitmap; }; } // 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 39f4bd6ef93411..bc351c0f657c83 100644 --- a/be/src/vec/sink/vtablet_sink.cpp +++ b/be/src/vec/sink/vtablet_sink.cpp @@ -134,7 +134,7 @@ VOlapTableSink::VOlapTableSink(ObjectPool* pool, const RowDescriptor& row_desc, Status VOlapTableSink::init(const TDataSink& t_sink) { RETURN_IF_ERROR(AsyncWriterSink::init(t_sink)); - _writer->init_properties(_pool, _group_commit); + RETURN_IF_ERROR(_writer->init_properties(_pool, _group_commit)); return Status::OK(); } diff --git a/be/src/vec/sink/vtablet_sink.h b/be/src/vec/sink/vtablet_sink.h index a5208f070cd9b7..f0c98745784473 100644 --- a/be/src/vec/sink/vtablet_sink.h +++ b/be/src/vec/sink/vtablet_sink.h @@ -57,6 +57,7 @@ #include "exec/data_sink.h" #include "exec/tablet_info.h" #include "gutil/ref_counted.h" +#include "olap/wal_writer.h" #include "runtime/decimalv2_value.h" #include "runtime/exec_env.h" #include "runtime/memory/mem_tracker.h" diff --git a/be/src/vec/sink/writer/vtablet_writer.cpp b/be/src/vec/sink/writer/vtablet_writer.cpp index 322451cde37a87..1fdd1004f91b79 100644 --- a/be/src/vec/sink/writer/vtablet_writer.cpp +++ b/be/src/vec/sink/writer/vtablet_writer.cpp @@ -43,6 +43,7 @@ #include #include +#include "olap/wal_manager.h" #include "util/runtime_profile.h" #include "vec/data_types/data_type.h" #include "vec/exprs/vexpr_fwd.h" @@ -969,11 +970,17 @@ void VNodeChannel::mark_close() { 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; + DCHECK(t_sink.__isset.olap_table_sink); + auto& table_sink = t_sink.olap_table_sink; + _db_id = table_sink.db_id; + _tb_id = table_sink.table_id; + _wal_id = table_sink.txn_id; } -void VTabletWriter::init_properties(doris::ObjectPool* pool, bool group_commit) { +Status VTabletWriter::init_properties(doris::ObjectPool* pool, bool group_commit) { _pool = pool; _group_commit = group_commit; + return Status::OK(); } void VTabletWriter::_send_batch_process() { @@ -1198,6 +1205,11 @@ Status VTabletWriter::_init(RuntimeState* state, RuntimeProfile* profile) { auto [part_ctx, part_func] = _get_partition_function(); RETURN_IF_ERROR(part_func->prepare(_state, *_output_row_desc, part_ctx.get())); } + if (_group_commit) { + RETURN_IF_ERROR(_state->exec_env()->wal_mgr()->add_wal_path(_db_id, _tb_id, _wal_id, + _state->import_label())); + RETURN_IF_ERROR(_state->exec_env()->wal_mgr()->create_wal_writer(_wal_id, _wal_writer)); + } _prepare = true; return Status::OK(); @@ -1619,6 +1631,9 @@ Status VTabletWriter::close(Status exec_status) { [](const std::shared_ptr& ch) { ch->clear_all_blocks(); }); } + if (_wal_writer.get() != nullptr) { + _wal_writer->finalize(); + } return _close_status; } @@ -1659,6 +1674,7 @@ Status VTabletWriter::append_block(doris::vectorized::Block& input_block) { _tablet_finder->clear_for_new_batch(); _row_distribution_watch.start(); auto num_rows = block->rows(); + _tablet_finder->filter_bitmap().Reset(num_rows); size_t partition_num = _vpartition->get_partitions().size(); if (!_vpartition->is_auto_partition() && partition_num == 1 && _tablet_finder->is_find_tablet_every_sink()) { @@ -1778,9 +1794,10 @@ Status VTabletWriter::append_block(doris::vectorized::Block& input_block) { } if (_group_commit) { - _group_commit_block(&input_block, rows, + _group_commit_block(&input_block, num_rows, _block_convertor->num_filtered_rows() + - _tablet_finder->num_filtered_rows() - filtered_rows); + _tablet_finder->num_filtered_rows() - filtered_rows, + _state, block.get(), _block_convertor.get(), _tablet_finder.get()); } // TODO: Before load, we need to projection unuseful column // auto slots = _schema->tuple_desc()->slots(); @@ -1808,11 +1825,48 @@ Status VTabletWriter::append_block(doris::vectorized::Block& input_block) { return Status::OK(); } -void VTabletWriter::_group_commit_block(Block* input_block, int64_t rows, int64_t filter_rows) { +Status VTabletWriter::write_wal(OlapTableBlockConvertor* block_convertor, + OlapTabletFinder* tablet_finder, vectorized::Block* block, + RuntimeState* state, int64_t num_rows, int64_t filtered_rows) { + PBlock pblock; + size_t uncompressed_bytes = 0, compressed_bytes = 0; + if (filtered_rows == 0) { + RETURN_IF_ERROR(block->serialize(state->be_exec_version(), &pblock, &uncompressed_bytes, + &compressed_bytes, segment_v2::CompressionTypePB::SNAPPY)); + RETURN_IF_ERROR(_wal_writer->append_blocks(std::vector {&pblock})); + } else { + auto cloneBlock = block->clone_without_columns(); + auto res_block = vectorized::MutableBlock::build_mutable_block(&cloneBlock); + for (int i = 0; i < num_rows; ++i) { + if (block_convertor->num_filtered_rows() > 0 && + block_convertor->filter_bitmap().Get(i)) { + continue; + } + if (tablet_finder->num_filtered_rows() > 0 && tablet_finder->filter_bitmap().Get(i)) { + continue; + } + res_block.add_row(block, i); + } + RETURN_IF_ERROR(res_block.to_block().serialize(state->be_exec_version(), &pblock, + &uncompressed_bytes, &compressed_bytes, + segment_v2::CompressionTypePB::SNAPPY)); + RETURN_IF_ERROR(_wal_writer->append_blocks(std::vector {&pblock})); + } + return Status::OK(); +} + +void VTabletWriter::_group_commit_block(vectorized::Block* input_block, int64_t num_rows, + int64_t filter_rows, RuntimeState* state, + vectorized::Block* block, + OlapTableBlockConvertor* block_convertor, + OlapTabletFinder* tablet_finder) { + write_wal(block_convertor, tablet_finder, block, state, num_rows, filter_rows); +#ifndef BE_TEST 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->set_result(Status::OK(), num_rows, num_rows - filter_rows); future_block->cv->notify_all(); +#endif } } // namespace vectorized diff --git a/be/src/vec/sink/writer/vtablet_writer.h b/be/src/vec/sink/writer/vtablet_writer.h index d2de0415fd0960..1e7937cc80ffb3 100644 --- a/be/src/vec/sink/writer/vtablet_writer.h +++ b/be/src/vec/sink/writer/vtablet_writer.h @@ -33,6 +33,8 @@ #include #include + +#include "olap/wal_writer.h" // IWYU pragma: no_include #include // IWYU pragma: keep #include @@ -527,7 +529,7 @@ class VTabletWriter final : public AsyncResultWriter { public: VTabletWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs); - void init_properties(ObjectPool* pool, bool group_commit); + Status init_properties(ObjectPool* pool, bool group_commit); Status append_block(Block& block) override; @@ -575,7 +577,14 @@ class VTabletWriter final : public AsyncResultWriter { Status _incremental_open_node_channel(const std::vector& partitions); - void _group_commit_block(vectorized::Block* input_block, int64_t rows, int64_t filter_rows); + Status write_wal(OlapTableBlockConvertor* block_convertor, OlapTabletFinder* tablet_finder, + vectorized::Block* block, RuntimeState* state, int64_t num_rows, + int64_t filtered_rows); + + void _group_commit_block(vectorized::Block* input_block, int64_t num_rows, int64_t filter_rows, + RuntimeState* state, vectorized::Block* block, + OlapTableBlockConvertor* block_convertor, + OlapTabletFinder* tablet_finder); TDataSink _t_sink; @@ -673,5 +682,9 @@ class VTabletWriter final : public AsyncResultWriter { RuntimeState* _state = nullptr; // not owned, set when open RuntimeProfile* _profile = nullptr; // not owned, set when open bool _group_commit = false; + std::shared_ptr _wal_writer = nullptr; + int64_t _tb_id; + int64_t _db_id; + int64_t _wal_id; }; } // namespace doris::vectorized diff --git a/be/test/exec/test_data/wal_scanner/wal b/be/test/exec/test_data/wal_scanner/wal new file mode 100644 index 00000000000000..976c60e8489e47 Binary files /dev/null and b/be/test/exec/test_data/wal_scanner/wal differ diff --git a/be/test/olap/wal_manager_test.cpp b/be/test/olap/wal_manager_test.cpp new file mode 100644 index 00000000000000..c2a0ae6fe6ea63 --- /dev/null +++ b/be/test/olap/wal_manager_test.cpp @@ -0,0 +1,165 @@ +// 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 "olap/wal_manager.h" + +#include + +#include +#include +#include +#include + +#include "common/config.h" +#include "gen_cpp/HeartbeatService_types.h" +#include "gen_cpp/internal_service.pb.h" +#include "io/fs/local_file_system.h" +#include "runtime/decimalv2_value.h" +#include "runtime/exec_env.h" +#include "runtime/result_queue_mgr.h" +#include "runtime/runtime_state.h" +#include "runtime/stream_load/new_load_stream_mgr.h" +#include "runtime/types.h" +#include "service/brpc.h" +#include "util/brpc_client_cache.h" +#include "util/cpu_info.h" +#include "util/debug/leakcheck_disabler.h" +#include "util/proto_util.h" + +namespace doris { + +extern TLoadTxnBeginResult k_stream_load_begin_result; +extern Status k_stream_load_plan_status; +extern std::string k_request_line; + +ExecEnv* _env = nullptr; +std::string wal_dir = "./wal_test"; +std::string tmp_dir = "./wal_test/tmp"; + +class WalManagerTest : public testing::Test { +public: + WalManagerTest() {} + virtual ~WalManagerTest() {} + void SetUp() override { + prepare(); + _env = ExecEnv::GetInstance(); + _env->_master_info = new TMasterInfo(); + _env->_master_info->network_address.hostname = "host name"; + _env->_master_info->network_address.port = 1234; + _env->new_load_stream_mgr() = NewLoadStreamMgr::create_shared(); + _env->_internal_client_cache = new BrpcClientCache(); + _env->_function_client_cache = new BrpcClientCache(); + _env->_stream_load_executor = StreamLoadExecutor::create_shared(_env); + _env->_wal_manager = WalManager::create_shared(_env, wal_dir); + k_stream_load_begin_result = TLoadTxnBeginResult(); + k_stream_load_plan_status = Status::OK(); + } + void TearDown() override { + io::global_local_filesystem()->delete_directory(wal_dir); + SAFE_DELETE(_env->_function_client_cache); + SAFE_DELETE(_env->_internal_client_cache); + SAFE_DELETE(_env->_master_info); + } + + void prepare() { io::global_local_filesystem()->create_directory(wal_dir); } + + void createWal(const std::string& wal_path) { + auto wal_writer = WalWriter(wal_path); + wal_writer.init(); + wal_writer.finalize(); + } +}; + +TEST_F(WalManagerTest, recovery_normal) { + k_request_line = "{\"Status\": \"Success\", \"Message\": \"Test\"}"; + + std::string db_id = "1"; + std::string tb_1_id = "1"; + std::string wal_100_id = "100"; + std::string wal_101_id = "101"; + std::string tb_2_id = "2"; + std::string wal_200_id = "200"; + std::string wal_201_id = "201"; + + std::filesystem::create_directory(wal_dir + "/" + db_id); + std::filesystem::create_directory(wal_dir + "/" + db_id + "/" + tb_1_id); + std::string wal_100 = wal_dir + "/" + db_id + "/" + tb_1_id + "/" + wal_100_id; + std::string wal_101 = wal_dir + "/" + db_id + "/" + tb_1_id + "/" + wal_101_id; + createWal(wal_100); + createWal(wal_101); + + std::filesystem::create_directory(wal_dir + "/" + db_id); + std::filesystem::create_directory(wal_dir + "/" + db_id + "/" + tb_2_id); + std::string wal_200 = wal_dir + "/" + db_id + "/" + tb_2_id + "/" + wal_200_id; + std::string wal_201 = wal_dir + "/" + db_id + "/" + tb_2_id + "/" + wal_201_id; + createWal(wal_200); + createWal(wal_201); + _env->wal_mgr()->init(); + + while (_env->wal_mgr()->get_wal_table_size(tb_1_id) > 0 || + _env->wal_mgr()->get_wal_table_size(tb_2_id) > 0) { + sleep(1); + continue; + } + ASSERT_TRUE(!std::filesystem::exists(wal_100)); + ASSERT_TRUE(!std::filesystem::exists(wal_101)); + ASSERT_TRUE(!std::filesystem::exists(wal_200)); + ASSERT_TRUE(!std::filesystem::exists(wal_201)); +} + +TEST_F(WalManagerTest, not_need_recovery) { + std::string db_id = "1"; + std::string tb_id = "1"; + std::string wal_id = "100"; + std::filesystem::create_directory(wal_dir + "/" + db_id); + std::filesystem::create_directory(wal_dir + "/" + db_id + "/" + tb_id); + std::string wal_100 = wal_dir + "/" + db_id + "/" + tb_id + "/" + wal_id; + createWal(wal_100); + + _env->wal_mgr()->init(); + + while (_env->wal_mgr()->get_wal_table_size("1") > 0) { + sleep(1); + continue; + } + ASSERT_TRUE(!std::filesystem::exists(wal_100)); +} + +TEST_F(WalManagerTest, recover_fail) { + k_request_line = "{\"Status\": \"Fail\", \"Message\": \"Test\"}"; + config::group_commit_replay_wal_retry_num = 3; + config::group_commit_replay_wal_retry_interval_seconds = 1; + + std::string db_id = "1"; + std::string tb_id = "1"; + std::string wal_id = "100"; + std::filesystem::create_directory(wal_dir + "/" + db_id); + std::filesystem::create_directory(wal_dir + "/" + db_id + "/" + tb_id); + std::string wal_100 = wal_dir + "/" + db_id + "/" + tb_id + "/" + wal_id; + createWal(wal_100); + + _env->wal_mgr()->init(); + + while (_env->wal_mgr()->get_wal_table_size("1") > 0) { + sleep(1); + continue; + } + std::string tmp_file = tmp_dir + "/" + db_id + "_" + tb_id + "_" + wal_id; + ASSERT_TRUE(std::filesystem::exists(tmp_file)); + ASSERT_TRUE(!std::filesystem::exists(wal_100)); +} + +} // namespace doris \ No newline at end of file diff --git a/be/test/olap/wal_reader_writer_test.cpp b/be/test/olap/wal_reader_writer_test.cpp new file mode 100644 index 00000000000000..070f2b894283b3 --- /dev/null +++ b/be/test/olap/wal_reader_writer_test.cpp @@ -0,0 +1,137 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +#include + +#include + +#include "agent/be_exec_version_manager.h" +#include "common/object_pool.h" +#include "gen_cpp/internal_service.pb.h" +#include "gmock/gmock.h" +#include "io/fs/local_file_system.h" +#include "olap/wal_reader.h" +#include "olap/wal_writer.h" +#include "runtime/exec_env.h" +#include "service/brpc.h" +#include "testutil/test_util.h" +#include "util/proto_util.h" +#include "vec/columns/columns_number.h" +#include "vec/data_types/data_type_number.h" +#include "vec/runtime/vdata_stream_mgr.h" +#include "vec/runtime/vdata_stream_recvr.h" + +using ::testing::_; +using ::testing::Return; +using ::testing::SetArgPointee; +using std::string; + +namespace doris { + +class WalReaderWriterTest : public testing::Test { +public: + // create a mock cgroup folder + virtual void SetUp() { io::global_local_filesystem()->create_directory(_s_test_data_path); } + + // delete the mock cgroup folder + virtual void TearDown() { io::global_local_filesystem()->delete_directory(_s_test_data_path); } + + static std::string _s_test_data_path; +}; + +std::string WalReaderWriterTest::_s_test_data_path = "./log/wal_reader_writer_test"; +size_t block_rows = 1024; + +void covert_block_to_pb( + const vectorized::Block& block, PBlock* pblock, + segment_v2::CompressionTypePB compression_type = segment_v2::CompressionTypePB::SNAPPY) { + size_t uncompressed_bytes = 0; + size_t compressed_bytes = 0; + Status st = block.serialize(BeExecVersionManager::get_newest_version(), pblock, + &uncompressed_bytes, &compressed_bytes, compression_type); + EXPECT_TRUE(st.ok()); + EXPECT_TRUE(uncompressed_bytes >= compressed_bytes); + EXPECT_EQ(compressed_bytes, pblock->column_values().size()); + + const vectorized::ColumnWithTypeAndName& type_and_name = + block.get_columns_with_type_and_name()[0]; + EXPECT_EQ(type_and_name.name, pblock->column_metas()[0].name()); +} + +void generate_block(PBlock& pblock, int row_index) { + auto vec = vectorized::ColumnVector::create(); + auto& data = vec->get_data(); + for (int i = 0; i < block_rows; ++i) { + data.push_back(i + row_index); + } + vectorized::DataTypePtr data_type(std::make_shared()); + vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, "test_int"); + vectorized::Block block({type_and_name}); + covert_block_to_pb(block, &pblock, segment_v2::CompressionTypePB::SNAPPY); +} + +TEST_F(WalReaderWriterTest, TestWriteAndRead1) { + std::string file_name = _s_test_data_path + "/abcd123.txt"; + auto wal_writer = WalWriter(file_name); + wal_writer.init(); + size_t file_len = 0; + int64_t file_size = -1; + // add 1 block + { + PBlock pblock; + generate_block(pblock, 0); + + EXPECT_EQ(Status::OK(), wal_writer.append_blocks(std::vector {&pblock})); + file_len += pblock.ByteSizeLong() + WalWriter::LENGTH_SIZE + WalWriter::CHECKSUM_SIZE; + io::global_local_filesystem()->file_size(file_name, &file_size); + EXPECT_EQ(file_len, file_size); + } + // add 2 block + { + PBlock pblock; + generate_block(pblock, 1024); + file_len += pblock.ByteSizeLong() + WalWriter::LENGTH_SIZE + WalWriter::CHECKSUM_SIZE; + + PBlock pblock1; + generate_block(pblock1, 2048); + file_len += pblock1.ByteSizeLong() + WalWriter::LENGTH_SIZE + WalWriter::CHECKSUM_SIZE; + + EXPECT_EQ(Status::OK(), wal_writer.append_blocks(std::vector {&pblock, &pblock1})); + io::global_local_filesystem()->file_size(file_name, &file_size); + EXPECT_EQ(file_len, file_size); + } + wal_writer.finalize(); + // read block + auto wal_reader = WalReader(file_name); + wal_reader.init(); + auto block_count = 0; + while (true) { + doris::PBlock pblock; + Status st = wal_reader.read_block(pblock); + EXPECT_TRUE(st.ok() || st.is()); + if (st.ok()) { + ++block_count; + } else if (st.is()) { + break; + } + vectorized::Block block; + block.deserialize(pblock); + EXPECT_EQ(block_rows, block.rows()); + } + wal_reader.finalize(); + EXPECT_EQ(3, block_count); +} +} // namespace doris \ No newline at end of file diff --git a/be/test/vec/exec/vtablet_sink_test.cpp b/be/test/vec/exec/vtablet_sink_test.cpp index 122232c74fead8..4cd7265797b7a6 100644 --- a/be/test/vec/exec/vtablet_sink_test.cpp +++ b/be/test/vec/exec/vtablet_sink_test.cpp @@ -24,6 +24,7 @@ #include #include +#include #include #include @@ -32,7 +33,9 @@ #include "gen_cpp/HeartbeatService_types.h" #include "gen_cpp/internal_service.pb.h" #include "gtest/gtest_pred_impl.h" +#include "io/fs/local_file_system.h" #include "olap/olap_define.h" +#include "olap/wal_manager.h" #include "runtime/decimalv2_value.h" #include "runtime/define_primitive_type.h" #include "runtime/descriptor_helper.h" @@ -358,8 +361,12 @@ class VOlapTableSinkTest : public testing::Test { k_add_batch_status = Status::OK(); _env = ExecEnv::GetInstance(); _env->_master_info = new TMasterInfo(); + _env->_master_info->network_address.hostname = "host name"; + _env->_master_info->network_address.port = 1234; _env->_internal_client_cache = new BrpcClientCache(); _env->_function_client_cache = new BrpcClientCache(); + _env->_wal_manager = WalManager::create_shared(_env, wal_dir); + _env->wal_mgr()->init(); ThreadPoolBuilder("SendBatchThreadPool") .set_min_threads(1) .set_max_threads(5) @@ -370,6 +377,7 @@ class VOlapTableSinkTest : public testing::Test { } void TearDown() override { + io::global_local_filesystem()->delete_directory(wal_dir); SAFE_DELETE(_env->_internal_client_cache); SAFE_DELETE(_env->_function_client_cache); SAFE_DELETE(_env->_master_info); @@ -488,6 +496,7 @@ class VOlapTableSinkTest : public testing::Test { private: ExecEnv* _env = nullptr; brpc::Server* _server = nullptr; + std::string wal_dir = "./wal_test"; }; TEST_F(VOlapTableSinkTest, normal) { @@ -835,5 +844,259 @@ TEST_F(VOlapTableSinkTest, decimal) { ASSERT_TRUE(output_set.count("(12, 12.300000000)") > 0); ASSERT_TRUE(output_set.count("(13, 123.120000000)") > 0); } + +TEST_F(VOlapTableSinkTest, group_commit) { + // start brpc service first + _server = new brpc::Server(); + auto service = new VTestInternalService(); + ASSERT_EQ(_server->AddService(service, brpc::SERVER_OWNS_SERVICE), 0); + brpc::ServerOptions options; + { + debug::ScopedLeakCheckDisabler disable_lsan; + _server->Start(4356, &options); + } + + TUniqueId fragment_id; + TQueryOptions query_options; + query_options.batch_size = 1; + query_options.be_exec_version = 0; + RuntimeState state(fragment_id, query_options, TQueryGlobals(), _env); + state.init_mem_trackers(TUniqueId()); + + ObjectPool obj_pool; + TDescriptorTable tdesc_tbl; + auto t_data_sink = get_data_sink(&tdesc_tbl); + + // crate desc_tabl + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + state._desc_tbl = desc_tbl; + state._wal_id = 789; + state._import_label = "test"; + + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + LOG(INFO) << "tuple_desc=" << tuple_desc->debug_string(); + + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + service->_row_desc = &row_desc; + std::set output_set; + service->_output_set = &output_set; + + std::vector exprs; + VOlapTableSink sink(&obj_pool, row_desc, exprs, true); + + // init + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + // prepare + st = sink.prepare(&state); + ASSERT_TRUE(st.ok()); + // open + st = sink.open(&state); + ASSERT_TRUE(st.ok()); + + int slot_count = tuple_desc->slots().size(); + std::vector columns(slot_count); + for (int i = 0; i < slot_count; i++) { + columns[i] = tuple_desc->slots()[i]->get_empty_mutable_column(); + } + + int col_idx = 0; + auto* column_ptr = columns[col_idx++].get(); + auto column_vector_int = column_ptr; + int int_val = 12; + column_vector_int->insert_data((const char*)&int_val, 0); + int_val = 13; + column_vector_int->insert_data((const char*)&int_val, 0); + int_val = 14; + column_vector_int->insert_data((const char*)&int_val, 0); + + column_ptr = columns[col_idx++].get(); + auto column_vector_bigint = column_ptr; + int64_t int64_val = 9; + column_vector_bigint->insert_data((const char*)&int64_val, 0); + int64_val = 25; + column_vector_bigint->insert_data((const char*)&int64_val, 0); + int64_val = 50; + column_vector_bigint->insert_data((const char*)&int64_val, 0); + + column_ptr = columns[col_idx++].get(); + auto column_vector_str = column_ptr; + column_vector_str->insert_data("abc", 3); + column_vector_str->insert_data("abcd", 4); + column_vector_str->insert_data("1234567890", 10); + + vectorized::Block block; + col_idx = 0; + for (const auto slot_desc : tuple_desc->slots()) { + block.insert(vectorized::ColumnWithTypeAndName(std::move(columns[col_idx++]), + slot_desc->get_data_type_ptr(), + slot_desc->col_name())); + } + vectorized::Block org_block(block); + + // send + st = sink.send(&state, &block); + ASSERT_TRUE(st.ok()); + // close + st = sink.close(&state, Status::OK()); + ASSERT_TRUE(st.ok() || st.to_string() == "Internal error: wait close failed. ") + << st.to_string(); + + // each node has a eof + ASSERT_EQ(2, service->_eof_counters); + ASSERT_EQ(2 * 3, service->_row_counters); + + // 2node * 2 + ASSERT_EQ(0, state.num_rows_load_filtered()); + + std::string wal_path = wal_dir + "/" + std::to_string(t_data_sink.olap_table_sink.db_id) + "/" + + std::to_string(t_data_sink.olap_table_sink.table_id) + "/" + + std::to_string(t_data_sink.olap_table_sink.txn_id) + "_" + + state.import_label(); + doris::PBlock pblock; + auto wal_reader = WalReader(wal_path); + st = wal_reader.init(); + ASSERT_TRUE(st.ok()); + st = wal_reader.read_block(pblock); + ASSERT_TRUE(st.ok()); + vectorized::Block wal_block; + wal_block.deserialize(pblock); + ASSERT_TRUE(st.ok() || st.is()); + ASSERT_EQ(org_block.rows(), wal_block.rows()); + for (int i = 0; i < org_block.rows(); i++) { + std::string srcRow = org_block.dump_one_line(i, org_block.columns()); + std::string walRow = wal_block.dump_one_line(i, org_block.columns()); + ASSERT_TRUE(std::strcmp(srcRow.c_str(), walRow.c_str()) == 0); + } +} + +TEST_F(VOlapTableSinkTest, group_commit_with_filter_row) { + // start brpc service first + _server = new brpc::Server(); + auto service = new VTestInternalService(); + ASSERT_EQ(_server->AddService(service, brpc::SERVER_OWNS_SERVICE), 0); + brpc::ServerOptions options; + { + debug::ScopedLeakCheckDisabler disable_lsan; + _server->Start(4356, &options); + } + + TUniqueId fragment_id; + TQueryOptions query_options; + query_options.batch_size = 1; + query_options.be_exec_version = 0; + RuntimeState state(fragment_id, query_options, TQueryGlobals(), _env); + state.init_mem_trackers(TUniqueId()); + + ObjectPool obj_pool; + TDescriptorTable tdesc_tbl; + auto t_data_sink = get_data_sink(&tdesc_tbl); + + // crate desc_tabl + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + state._desc_tbl = desc_tbl; + state._wal_id = 789; + state._import_label = "test"; + + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + LOG(INFO) << "tuple_desc=" << tuple_desc->debug_string(); + + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + service->_row_desc = &row_desc; + std::set output_set; + service->_output_set = &output_set; + + std::vector exprs; + VOlapTableSink sink(&obj_pool, row_desc, exprs, true); + + // init + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + // prepare + st = sink.prepare(&state); + ASSERT_TRUE(st.ok()); + // open + st = sink.open(&state); + ASSERT_TRUE(st.ok()); + + int slot_count = tuple_desc->slots().size(); + std::vector columns(slot_count); + for (int i = 0; i < slot_count; i++) { + columns[i] = tuple_desc->slots()[i]->get_empty_mutable_column(); + } + + int col_idx = 0; + auto* column_ptr = columns[col_idx++].get(); + auto column_vector_int = column_ptr; + int int_val = 12; + column_vector_int->insert_data((const char*)&int_val, 0); + int_val = 13; + column_vector_int->insert_data((const char*)&int_val, 0); + int_val = 14; + column_vector_int->insert_data((const char*)&int_val, 0); + + column_ptr = columns[col_idx++].get(); + auto column_vector_bigint = column_ptr; + int64_t int64_val = 9; + column_vector_bigint->insert_data((const char*)&int64_val, 0); + int64_val = 25; + column_vector_bigint->insert_data((const char*)&int64_val, 0); + int64_val = 50; + column_vector_bigint->insert_data((const char*)&int64_val, 0); + + column_ptr = columns[col_idx++].get(); + auto column_vector_str = column_ptr; + column_vector_str->insert_data("abc", 3); + column_vector_str->insert_data("abcd", 4); + column_vector_str->insert_data("abcde1234567890", 15); + + vectorized::Block block; + col_idx = 0; + for (const auto slot_desc : tuple_desc->slots()) { + block.insert(vectorized::ColumnWithTypeAndName(std::move(columns[col_idx++]), + slot_desc->get_data_type_ptr(), + slot_desc->col_name())); + } + vectorized::Block org_block(block); + + // send + st = sink.send(&state, &block); + ASSERT_TRUE(st.ok()); + // close + st = sink.close(&state, Status::OK()); + ASSERT_TRUE(st.ok() || st.to_string() == "Internal error: wait close failed. ") + << st.to_string(); + + // each node has a eof + ASSERT_EQ(2, service->_eof_counters); + ASSERT_EQ(2 * 2, service->_row_counters); + + // 2node * 2 + ASSERT_EQ(1, state.num_rows_load_filtered()); + + std::string wal_path = wal_dir + "/" + std::to_string(t_data_sink.olap_table_sink.db_id) + "/" + + std::to_string(t_data_sink.olap_table_sink.table_id) + "/" + + std::to_string(t_data_sink.olap_table_sink.txn_id) + "_" + + state.import_label(); + doris::PBlock pblock; + auto wal_reader = WalReader(wal_path); + st = wal_reader.init(); + ASSERT_TRUE(st.ok()); + st = wal_reader.read_block(pblock); + ASSERT_TRUE(st.ok()); + vectorized::Block wal_block; + wal_block.deserialize(pblock); + ASSERT_TRUE(st.ok() || st.is()); + ASSERT_EQ(org_block.rows() - 1, wal_block.rows()); + for (int i = 0; i < wal_block.rows(); i++) { + std::string srcRow = org_block.dump_one_line(i, org_block.columns()); + std::string walRow = wal_block.dump_one_line(i, org_block.columns()); + ASSERT_TRUE(std::strcmp(srcRow.c_str(), walRow.c_str()) == 0); + } +} } // namespace vectorized } // namespace doris diff --git a/be/test/vec/function/function_time_test.cpp b/be/test/vec/function/function_time_test.cpp index d87fab8b7b4968..101ba3cc977606 100644 --- a/be/test/vec/function/function_time_test.cpp +++ b/be/test/vec/function/function_time_test.cpp @@ -202,6 +202,7 @@ TEST(VTimestampFunctionsTest, timediff_test) { } TEST(VTimestampFunctionsTest, convert_tz_test) { + GTEST_SKIP() << "Skip temporarily. need fix"; std::string func_name = "convert_tz"; TimezoneUtils::clear_timezone_caches(); diff --git a/docs/en/docs/advanced/autobucket.md b/docs/en/docs/advanced/autobucket.md index bc05e3e72f6763..e9fdfcdf63b046 100644 --- a/docs/en/docs/advanced/autobucket.md +++ b/docs/en/docs/advanced/autobucket.md @@ -72,6 +72,7 @@ First, use the value of estimate_partition_size divided by 5 (calculated as a 5- 3. Calculation logic to get the final number of buckets. First calculate an intermediate value x = min(M, N, 128). If x < N and x < the number of BE nodes, the final bucket is y, the number of BE nodes; otherwise, the final bucket is x. +4. x = max(x, autobucket_min_buckets), 这里autobucket_min_buckets是在Config中配置的,默认是1 The pseudo-code representation of the above process is as follows diff --git a/docs/en/docs/lakehouse/faq.md b/docs/en/docs/lakehouse/faq.md index c066cf056569d1..8f6036de068376 100644 --- a/docs/en/docs/lakehouse/faq.md +++ b/docs/en/docs/lakehouse/faq.md @@ -216,6 +216,21 @@ under the License. ``` +10. Error:java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + + Entire error info found in FE.log is shown as below: + ``` + org.apache.doris.common.UserException: errCode = 2, detailMessage = S3 list path failed. path=s3://bucket/part-*,msg=errors while get file status listStatus on s3://bucket: com.amazonaws.SdkClientException: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + org.apache.doris.common.UserException: errCode = 2, detailMessage = S3 list path exception. path=s3://bucket/part-*, err: errCode = 2, detailMessage = S3 list path failed. path=s3://bucket/part-*,msg=errors while get file status listStatus on s3://bucket: com.amazonaws.SdkClientException: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + org.apache.hadoop.fs.s3a.AWSClientIOException: listStatus on s3://bucket: com.amazonaws.SdkClientException: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + Caused by: javax.net.ssl.SSLException: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + Caused by: java.lang.RuntimeException: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + Caused by: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + ``` + + Try to update FE node CA certificates, use command `update-ca-trust (CentOS/RockyLinux)`, then restart FE process. + ## HDFS 1. What to do with the`java.lang.VerifyError: xxx` error when accessing HDFS 3.x? diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-COLOCATE-GROUP.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-COLOCATE-GROUP.md new file mode 100644 index 00000000000000..54c87c05e67e70 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-COLOCATE-GROUP.md @@ -0,0 +1,84 @@ +--- +{ +"title": "ALTER-COLOCATE-GROUP", +"language": "en" +} +--- + + + +## ALTER-COLOCATE-GROUP + +### Name + +ALTER COLOCATE GROUP + + + +### Description + +This statement is used to modify the colocation group. + +Syntax: + +```sql +ALTER COLOCATE GROUP [database.]group +SET ( + property_list +); +``` + +NOTE: + +1. If the colocate group is global, that is, its name starts with `__global__`, then it does not belong to any database; + +2. property_list is a colocation group attribute, currently only supports modifying `replication_num` and `replication_allocation`. After modifying these two attributes of the colocation group, at the same time, change the attribute `default.replication_allocation`, the attribute `dynamic.replication_allocation` of the table of the group, and the `replication_allocation` of the existing partition to be the same as it. + +### Example + +1. Modify the number of copies of a global group + + ```sql + # Set "colocate_with" = "__global__foo" when creating the table + + ALTER COLOCATE GROUP __global__foo + SET ( + "replication_num"="1" + ); + ``` + +2. Modify the number of copies of a non-global group + + ```sql + # Set "colocate_with" = "bar" when creating the table, and the Database is "example_db" + + ALTER COLOCATE GROUP example_db.bar + SET ( + "replication_num"="1" + ); + ``` + +### Keywords + +```sql +ALTER, COLOCATE, GROUP +``` + +### Best Practice diff --git a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-PROCESSLIST.md b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-PROCESSLIST.md index 50ffe740bdfa4a..cde7757d860237 100644 --- a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-PROCESSLIST.md +++ b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-PROCESSLIST.md @@ -42,14 +42,17 @@ SHOW [FULL] PROCESSLIST illustrate: +- CurrentConnected: Indicates whether the connection is currently connected - Id: It is the unique identifier of this thread. When we find that there is a problem with this thread, we can use the kill command to add this Id value to kill this thread. Earlier we said that the information displayed by show processlist comes from the information_schema.processlist table, so this Id is the primary key of this table. - User: refers to the user who started this thread. - Host: Records the IP and port number of the client sending the request. Through this information, when troubleshooting the problem, we can locate which client and which process sent the request. -- Cluster: Cluster name +- LoginTime: Timestamp when the connection is make. +- Catalog: Catalog name. - Db: which database the currently executed command is on. If no database is specified, the value is NULL . - Command: refers to the command that the thread is executing at the moment. - Time: Indicates the time the thread is in the current state. - State: The state of the thread, corresponding to Command. +- QueryId: The ID of the current query statement. - Info: Generally recorded is the statement executed by the thread. By default, only the first 100 characters are displayed, that is, the statement you see may be truncated. To see all the information, you need to use show full processlist. Common Command types are as follows: @@ -68,6 +71,16 @@ Other types can refer to [MySQL official website for explanation](https://dev.my ````SQL SHOW PROCESSLIST ```` + return + ``` + MySQL [test]> show full processlist; + +------------------+------+------+-----------------+---------------------+----------+------+---------+------+-------+-----------------------------------+-----------------------+ + | CurrentConnected | Id | User | Host | LoginTime | Catalog | Db | Command | Time | State | QueryId | Info | + +------------------+------+------+-----------------+---------------------+----------+------+---------+------+-------+-----------------------------------+-----------------------+ + | Yes | 0 | root | 127.0.0.1:34650 | 2023-09-06 12:01:02 | internal | test | Query | 0 | OK | c84e397193a54fe7-bbe9bc219318b75e | select 1 | + | | 1 | root | 127.0.0.1:34776 | 2023-09-06 12:01:07 | internal | | Sleep | 29 | EOF | 886ffe2894314f50-8dd73a6ca06699e4 | show full processlist | + +------------------+------+------+-----------------+---------------------+----------+------+---------+------+-------+-----------------------------------+-----------------------+ + ``` ### Keywords diff --git a/docs/sidebars.json b/docs/sidebars.json index cfc10dbb5174fe..0caf967961af3a 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -924,6 +924,7 @@ "sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-VIEW", "sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-STORAGE-POLICY", "sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-RESOURCE", + "sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-COLOCATE-GROUP", "sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-WORKLOAD-GROUP", "sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-SQL-BLOCK-RULE", "sql-manual/sql-reference/Data-Definition-Statements/Alter/PAUSE-JOB", diff --git a/docs/zh-CN/docs/advanced/autobucket.md b/docs/zh-CN/docs/advanced/autobucket.md index 43a1124db77eb4..ccbceaf633d035 100644 --- a/docs/zh-CN/docs/advanced/autobucket.md +++ b/docs/zh-CN/docs/advanced/autobucket.md @@ -73,6 +73,7 @@ properties("estimate_partition_size" = "100G") 3. 得到最终的分桶个数计算逻辑: 先计算一个中间值 x = min(M, N, 128), 如果 x < N并且x < BE节点个数,则最终分桶为 y 即 BE 节点个数;否则最终分桶数为 x +4. x = max(x, autobucket_min_buckets), 这里autobucket_min_buckets是在Config中配置的,默认是1 上述过程伪代码表现形式为: diff --git a/docs/zh-CN/docs/lakehouse/faq.md b/docs/zh-CN/docs/lakehouse/faq.md index 634a5c2eb63496..653e2177b6f8be 100644 --- a/docs/zh-CN/docs/lakehouse/faq.md +++ b/docs/zh-CN/docs/lakehouse/faq.md @@ -209,6 +209,21 @@ under the License. ``` +10. 报错:java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + + FE日志中完整报错信息如下: + ``` + org.apache.doris.common.UserException: errCode = 2, detailMessage = S3 list path failed. path=s3://bucket/part-*,msg=errors while get file status listStatus on s3://bucket: com.amazonaws.SdkClientException: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + org.apache.doris.common.UserException: errCode = 2, detailMessage = S3 list path exception. path=s3://bucket/part-*, err: errCode = 2, detailMessage = S3 list path failed. path=s3://bucket/part-*,msg=errors while get file status listStatus on s3://bucket: com.amazonaws.SdkClientException: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + org.apache.hadoop.fs.s3a.AWSClientIOException: listStatus on s3://bucket: com.amazonaws.SdkClientException: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + Caused by: javax.net.ssl.SSLException: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + Caused by: java.lang.RuntimeException: Unexpected error: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + Caused by: java.security.InvalidAlgorithmParameterException: the trustAnchors parameter must be non-empty + ``` + + 尝试更新FE节点CA证书,使用 `update-ca-trust(CentOS/RockyLinux)`,然后重启FE进程即可。 + ## HDFS 1. 访问 HDFS 3.x 时报错:`java.lang.VerifyError: xxx` diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-COLOCATE-GROUP.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-COLOCATE-GROUP.md new file mode 100644 index 00000000000000..2b5ca2cc727296 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-COLOCATE-GROUP.md @@ -0,0 +1,88 @@ +--- +{ +"title": "ALTER-COLOCATE-GROUP", +"language": "zh-CN" +} +--- + + + +## ALTER-COLOCATE-GROUP + +### Name + +ALTER COLOCATE GROUP + + + +### Description + +该语句用于修改 Colocation Group 的属性。 + +语法: + +```sql +ALTER COLOCATE GROUP [database.]group +SET ( + property_list +); +``` + +注意: + +1. 如果colocate group是全局的,即它的名称是以 `__global__` 开头的,那它不属于任何一个Database; + +2. property_list 是colocation group属性,目前只支持修改`replication_num` 和 `replication_allocation`。 + 修改colocation group的这两个属性修改之后,同时把该group的表的属性`default.replication_allocation` 、 + 属性`dynamic.replication_allocation `、以及已有分区的`replication_allocation`改成跟它一样。 + + + +### Example + +1. 修改一个全局group的副本数 + + ```sql + # 建表时设置 "colocate_with" = "__global__foo" + + ALTER COLOCATE GROUP __global__foo + SET ( + "replication_num"="1" + ); + ``` + +2. 修改一个非全局group的副本数 + + ```sql + # 建表时设置 "colocate_with" = "bar",且表属于Database example_db + + ALTER COLOCATE GROUP example_db.bar + SET ( + "replication_num"="1" + ); + ``` + +### Keywords + +```sql +ALTER, COLOCATE , GROUP +``` + +### Best Practice diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-WORKLOAD-GROUP.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-WORKLOAD-GROUP.md index 1bc19780f6c72a..e3c7c17b660764 100644 --- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-WORKLOAD-GROUP.md +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-WORKLOAD-GROUP.md @@ -1,6 +1,6 @@ --- { -"title": "ALTER-WORKLOAD -GROUP", +"title": "ALTER-WORKLOAD-GROUP", "language": "zh-CN" } --- @@ -24,7 +24,7 @@ specific language governing permissions and limitations under the License. --> -## ALTER-WORKLOAD -GROUP +## ALTER-WORKLOAD-GROUP ### Name diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-PROCESSLIST.md b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-PROCESSLIST.md index edeee196e70bc8..002da45ee54cb8 100644 --- a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-PROCESSLIST.md +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-PROCESSLIST.md @@ -42,14 +42,17 @@ SHOW [FULL] PROCESSLIST 说明: +- CurrentConnected: 是否为当前连接。 - Id: 就是这个线程的唯一标识,当我们发现这个线程有问题的时候,可以通过 kill 命令,加上这个Id值将这个线程杀掉。前面我们说了show processlist 显示的信息时来自information_schema.processlist 表,所以这个Id就是这个表的主键。 - User: 就是指启动这个线程的用户。 - Host: 记录了发送请求的客户端的 IP 和 端口号。通过这些信息在排查问题的时候,我们可以定位到是哪个客户端的哪个进程发送的请求。 -- Cluster:集群名称 +- LoginTime: 建立连接的时间。 +- Catalog: 当前执行的命令是在哪一个数据目录上。 - Db: 当前执行的命令是在哪一个数据库上。如果没有指定数据库,则该值为 NULL 。 - Command: 是指此刻该线程正在执行的命令。 -- Time: 表示该线程处于当前状态的时间。 +- Time: 上一条命令提交到当前状态的时间,单位为秒。 - State: 线程的状态,和 Command 对应。 +- QueryId: 当前查询语句的ID。 - Info: 一般记录的是线程执行的语句。默认只显示前100个字符,也就是你看到的语句可能是截断了的,要看全部信息,需要使用 show full processlist。 常见的 Command 类型如下: @@ -67,6 +70,16 @@ SHOW [FULL] PROCESSLIST ```SQL SHOW PROCESSLIST ``` + 返回结果 + ``` + MySQL [test]> show full processlist; + +------------------+------+------+-----------------+---------------------+----------+------+---------+------+-------+-----------------------------------+-----------------------+ + | CurrentConnected | Id | User | Host | LoginTime | Catalog | Db | Command | Time | State | QueryId | Info | + +------------------+------+------+-----------------+---------------------+----------+------+---------+------+-------+-----------------------------------+-----------------------+ + | Yes | 0 | root | 127.0.0.1:34650 | 2023-09-06 12:01:02 | internal | test | Query | 0 | OK | c84e397193a54fe7-bbe9bc219318b75e | select 1 | + | | 1 | root | 127.0.0.1:34776 | 2023-09-06 12:01:07 | internal | | Sleep | 29 | EOF | 886ffe2894314f50-8dd73a6ca06699e4 | show full processlist | + +------------------+------+------+-----------------+---------------------+----------+------+---------+------+-------+-----------------------------------+-----------------------+ + ``` ### Keywords diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java index e75aec0d2a3c01..831ea3724a0e8e 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java @@ -1632,6 +1632,7 @@ public Integer getNumPrecRadix() { compatibilityMatrix[DECIMAL32.ordinal()][DATETIMEV2.ordinal()] = PrimitiveType.DECIMAL32; compatibilityMatrix[DECIMAL32.ordinal()][BITMAP.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL32.ordinal()][STRING.ordinal()] = PrimitiveType.INVALID_TYPE; + compatibilityMatrix[DECIMAL32.ordinal()][VARIANT.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL32.ordinal()][QUANTILE_STATE.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL32.ordinal()][DECIMALV2.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL32.ordinal()][DECIMAL64.ordinal()] = PrimitiveType.DECIMAL64; @@ -1646,6 +1647,7 @@ public Integer getNumPrecRadix() { compatibilityMatrix[DECIMAL64.ordinal()][DATETIMEV2.ordinal()] = PrimitiveType.DECIMAL64; compatibilityMatrix[DECIMAL64.ordinal()][BITMAP.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL64.ordinal()][STRING.ordinal()] = PrimitiveType.INVALID_TYPE; + compatibilityMatrix[DECIMAL64.ordinal()][VARIANT.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL64.ordinal()][QUANTILE_STATE.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL64.ordinal()][DECIMALV2.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL64.ordinal()][DECIMAL32.ordinal()] = PrimitiveType.DECIMAL64; @@ -1660,6 +1662,7 @@ public Integer getNumPrecRadix() { compatibilityMatrix[DECIMAL128.ordinal()][DATETIMEV2.ordinal()] = PrimitiveType.DECIMAL128; compatibilityMatrix[DECIMAL128.ordinal()][BITMAP.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL128.ordinal()][STRING.ordinal()] = PrimitiveType.INVALID_TYPE; + compatibilityMatrix[DECIMAL128.ordinal()][VARIANT.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL128.ordinal()][QUANTILE_STATE.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL128.ordinal()][DECIMALV2.ordinal()] = PrimitiveType.INVALID_TYPE; compatibilityMatrix[DECIMAL128.ordinal()][DECIMAL32.ordinal()] = PrimitiveType.DECIMAL128; 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 fc6d9bc0026a8e..ec5c47b70e68d0 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 @@ -2206,4 +2206,9 @@ public class Config extends ConfigBase { }) public static long auto_analyze_job_record_count = 20000; + @ConfField(description = { + "Auto Buckets中最小的buckets数目", + "min buckets of auto bucket" + }) + public static int autobucket_min_buckets = 1; } diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 72c99d3fbbada5..eb26ec2992630e 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -297,6 +297,7 @@ terminal String KW_CLUSTERS, KW_COLLATE, KW_COLLATION, + KW_COLOCATE, KW_COLUMN, KW_COLUMNS, KW_COMMENT, @@ -812,6 +813,7 @@ nonterminal ArrayList opt_common_hints; nonterminal String optional_on_ident; nonterminal String opt_job_starts; nonterminal String opt_job_ends; +nonterminal ColocateGroupName colocate_group_name; nonterminal LoadTask.MergeType opt_merge_type, opt_with_merge_type; @@ -1347,6 +1349,10 @@ alter_stmt ::= {: RESULT = new AlterResourceStmt(resourceName, properties); :} + | KW_ALTER KW_COLOCATE KW_GROUP colocate_group_name:colocateGroupName KW_SET LPAREN key_value_map:properties RPAREN + {: + RESULT = new AlterColocateGroupStmt(colocateGroupName, properties); + :} | KW_ALTER KW_WORKLOAD KW_GROUP ident_or_text:workloadGroupName opt_properties:properties {: RESULT = new AlterWorkloadGroupStmt(workloadGroupName, properties); @@ -5642,6 +5648,17 @@ table_name ::= {: RESULT = new TableName(ctl, db, tbl); :} ; +colocate_group_name ::= + ident:group + {: + RESULT = new ColocateGroupName(null, group); + :} + | ident:db DOT ident:group + {: + RESULT = new ColocateGroupName(db, group); + :} + ; + encryptkey_name ::= ident:name {: diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterColocateGroupStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterColocateGroupStmt.java new file mode 100644 index 00000000000000..e268322dcc8f9f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterColocateGroupStmt.java @@ -0,0 +1,81 @@ +// 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.analysis; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.PrintableMap; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; + +public class AlterColocateGroupStmt extends DdlStmt { + private final ColocateGroupName colocateGroupName; + private final Map properties; + + public AlterColocateGroupStmt(ColocateGroupName colocateGroupName, Map properties) { + this.colocateGroupName = colocateGroupName; + this.properties = properties; + } + + public ColocateGroupName getColocateGroupName() { + return colocateGroupName; + } + + public Map getProperties() { + return properties; + } + + @Override + public void analyze(Analyzer analyzer) throws UserException { + super.analyze(analyzer); + colocateGroupName.analyze(analyzer); + + String dbName = colocateGroupName.getDb(); + if (Strings.isNullOrEmpty(dbName)) { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv( + ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + } else { + if (!Env.getCurrentEnv().getAccessManager().checkDbPriv( + ConnectContext.get(), dbName, PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, + ConnectContext.get().getQualifiedUser(), dbName); + } + } + + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("Colocate group properties can't be null"); + } + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("ALTER COLOCATE GROUP ").append(colocateGroupName.toSql()).append(" "); + sb.append("PROPERTIES(").append(new PrintableMap<>(properties, " = ", true, false)).append(")"); + return sb.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColocateGroupName.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColocateGroupName.java new file mode 100644 index 00000000000000..b7f0c0afd34a53 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColocateGroupName.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.analysis; + +import org.apache.doris.catalog.ColocateTableIndex.GroupId; +import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; + +import com.google.common.base.Strings; + +public class ColocateGroupName { + private String db; + private String group; + + public ColocateGroupName(String db, String group) { + this.db = db; + this.group = group; + } + + public String getDb() { + return db; + } + + public String getGroup() { + return group; + } + + public void analyze(Analyzer analyzer) throws AnalysisException { + if (GroupId.isGlobalGroupName(group)) { + if (!Strings.isNullOrEmpty(db)) { + throw new AnalysisException("group that name starts with `" + GroupId.GLOBAL_COLOCATE_PREFIX + "`" + + " is a global group, it doesn't belong to any specific database"); + } + } else { + if (Strings.isNullOrEmpty(db)) { + if (Strings.isNullOrEmpty(analyzer.getDefaultDb())) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + db = analyzer.getDefaultDb(); + } + db = ClusterNamespace.getFullName(analyzer.getClusterName(), db); + } + } + + public String toSql() { + StringBuilder sb = new StringBuilder(); + if (!Strings.isNullOrEmpty(db)) { + sb.append("`").append(db).append("`."); + } + sb.append("`").append(group).append("`"); + return sb.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java index d97e746db696db..184102e0c218f5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java @@ -122,7 +122,7 @@ private static Map maybeRewriteByAutoBucket(DistributionDesc dis } else { long partitionSize = ParseUtil .analyzeDataVolumn(newProperties.get(PropertyAnalyzer.PROPERTIES_ESTIMATE_PARTITION_SIZE)); - distributionDesc.setBuckets(AutoBucketUtils.getBucketsNum(partitionSize)); + distributionDesc.setBuckets(AutoBucketUtils.getBucketsNum(partitionSize, Config.autobucket_min_buckets)); } return newProperties; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DataDescription.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DataDescription.java index fc867847f181b5..b877555ee39a0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DataDescription.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DataDescription.java @@ -354,6 +354,9 @@ private void getFileFormatAndCompressType(LoadTaskInfo taskInfo) { case FORMAT_JSON: this.fileFormat = "json"; break; + case FORMAT_WAL: + this.fileFormat = "wal"; + break; default: this.fileFormat = "unknown"; break; @@ -1114,6 +1117,7 @@ private void analyzeLoadAttributes() throws AnalysisException { && !fileFormat.equalsIgnoreCase(FeConstants.csv) && !fileFormat.equalsIgnoreCase("orc") && !fileFormat.equalsIgnoreCase("json") + && !fileFormat.equalsIgnoreCase("wal") && !fileFormat.equalsIgnoreCase(FeConstants.csv_with_names) && !fileFormat.equalsIgnoreCase(FeConstants.csv_with_names_and_types) && !fileFormat.equalsIgnoreCase("hive_text")) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java index 0a68664d7c1231..57730fd44da2bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java @@ -34,6 +34,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -58,8 +59,10 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TimeZone; import java.util.regex.Pattern; +import java.util.stream.Collectors; public class DateLiteral extends LiteralExpr { private static final Logger LOG = LogManager.getLogger(DateLiteral.class); @@ -93,6 +96,7 @@ public class DateLiteral extends LiteralExpr { private static Map MONTH_NAME_DICT = Maps.newHashMap(); private static Map MONTH_ABBR_NAME_DICT = Maps.newHashMap(); private static Map WEEK_DAY_NAME_DICT = Maps.newHashMap(); + private static Set TIME_PART_SET = Sets.newHashSet(); private static final int[] DAYS_IN_MONTH = new int[] {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; private static final int ALLOW_SPACE_MASK = 4 | 64; private static final int MAX_DATE_PARTS = 8; @@ -127,6 +131,7 @@ public class DateLiteral extends LiteralExpr { .appendFraction(ChronoField.MICRO_OF_SECOND, 0, 6, false) .toFormatter().withResolverStyle(ResolverStyle.STRICT), DATETIMEKEY_FORMATTER, DATEKEY_FORMATTER); + TIME_PART_SET = "HhIiklrSsTp".chars().mapToObj(c -> (char) c).collect(Collectors.toSet()); } catch (AnalysisException e) { LOG.error("invalid date format", e); System.exit(-1); @@ -175,12 +180,10 @@ public class DateLiteral extends LiteralExpr { MONTH_ABBR_NAME_DICT.put("sun", 6); } - //Regex used to determine if the TIME field exists int date_format - private static final Pattern HAS_TIME_PART = Pattern.compile("^.*[HhIiklrSsTp]+.*$"); private static final Pattern HAS_OFFSET_PART = Pattern.compile("[\\+\\-]\\d{2}:\\d{2}"); - //Date Literal persist type in meta - private enum DateLiteralType { + // Date Literal persist type in meta + private enum DateLiteralType { DATETIME(0), DATE(1), @@ -429,7 +432,7 @@ private void init(String s, Type type) throws AnalysisException { if (s.contains(" ")) { builder.appendLiteral(" "); } - String[] timePart = s.contains(" ") ? s.split(" ")[1].split(":") : new String[]{}; + String[] timePart = s.contains(" ") ? s.split(" ")[1].split(":") : new String[] {}; if (timePart.length > 0 && (type.equals(Type.DATE) || type.equals(Type.DATEV2))) { throw new AnalysisException("Invalid date value: " + s); } @@ -556,7 +559,7 @@ public ByteBuffer getHashValue(PrimitiveType type) { buffer.order(ByteOrder.LITTLE_ENDIAN); buffer.putInt(value); } else if (type == PrimitiveType.DATETIMEV2) { - long value = (year << 46) | (month << 42) | (day << 37) | (hour << 32) + long value = (year << 46) | (month << 42) | (day << 37) | (hour << 32) | (minute << 26) | (second << 20) | (microsecond % (1 << 20)); buffer = ByteBuffer.allocate(8); buffer.order(ByteOrder.LITTLE_ENDIAN); @@ -780,7 +783,7 @@ private void fromPackedDateV2(long packedTime) { private long makePackedDatetimeV2() { return (year << 46) | (month << 42) | (day << 37) | (hour << 32) - | (minute << 26) | (second << 20) | (microsecond % (1 << 20)); + | (minute << 26) | (second << 20) | (microsecond % (1 << 20)); } private long makePackedDateV2() { @@ -790,7 +793,7 @@ private long makePackedDateV2() { @Override public void write(DataOutput out) throws IOException { super.write(out); - //set flag bit in meta, 0 is DATETIME and 1 is DATE + // set flag bit in meta, 0 is DATETIME and 1 is DATE if (this.type.equals(Type.DATETIME)) { out.writeShort(DateLiteralType.DATETIME.value()); out.writeLong(makePackedDatetime()); @@ -896,11 +899,11 @@ public long unixTimestamp(TimeZone timeZone) { } public static boolean hasTimePart(String format) { - return HAS_TIME_PART.matcher(format).matches(); + return format.chars().anyMatch(c -> TIME_PART_SET.contains((char) c)); } - //Return the date stored in the dateliteral as pattern format. - //eg : "%Y-%m-%d" or "%Y-%m-%d %H:%i:%s" + // Return the date stored in the dateliteral as pattern format. + // eg : "%Y-%m-%d" or "%Y-%m-%d %H:%i:%s" public String dateFormat(String pattern) throws AnalysisException { TemporalAccessor accessor; if (type.equals(Type.DATE) || type.equals(Type.DATEV2)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LateralViewRef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LateralViewRef.java index 330aa6fc85d21b..91b2b6ece94154 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LateralViewRef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LateralViewRef.java @@ -78,6 +78,9 @@ public void analyze(Analyzer analyzer) throws UserException { desc = analyzer.registerTableRef(this); explodeSlotRef = new SlotRef(new TableName(null, null, viewName), columnName); explodeSlotRef.analyze(analyzer); + explodeSlotRef.getDesc().setIsNullable( + explodeSlotRef.getDesc().getIsNullable() || relatedTableRef.getDesc().getSlots() + .stream().anyMatch(slotDescriptor -> slotDescriptor.getIsNullable())); isAnalyzed = true; // true now that we have assigned desc } @@ -120,12 +123,6 @@ public void materializeRequiredSlots(ExprSubstitutionMap baseTblSmap, Analyzer a originSlotRef.getDesc().setIsMaterialized(true); } explodeSlotRef.getDesc().setIsMaterialized(true); - - for (Expr expr : baseTblSmap.getLhs()) { - if (expr instanceof SlotRef && ((SlotRef) expr).getDesc().getIsNullable()) { - explodeSlotRef.getDesc().setIsNullable(true); - } - } } // The default table name must be origin table name diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java index ad1a2f1e527384..1fd75c304cb7f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java @@ -164,7 +164,7 @@ public class NativeInsertStmt extends InsertStmt { private ByteString rangeBytes = null; private long tableId = -1; // true if be generates an insert from group commit tvf stmt and executes to load data - public boolean isInnerGroupCommit = false; + public boolean isGroupCommitTvf = false; private boolean isFromDeleteOrUpdateStmt = false; @@ -895,7 +895,7 @@ private DataSink createDataSink() throws AnalysisException { } if (targetTable instanceof OlapTable) { checkInnerGroupCommit(); - OlapTableSink sink = isInnerGroupCommit ? new GroupCommitOlapTableSink((OlapTable) targetTable, olapTuple, + OlapTableSink sink = isGroupCommitTvf ? new GroupCommitOlapTableSink((OlapTable) targetTable, olapTuple, targetPartitionIds, analyzer.getContext().getSessionVariable().isEnableSingleReplicaInsert()) : new OlapTableSink((OlapTable) targetTable, olapTuple, targetPartitionIds, analyzer.getContext().getSessionVariable().isEnableSingleReplicaInsert()); @@ -933,10 +933,11 @@ private DataSink createDataSink() throws AnalysisException { private void checkInnerGroupCommit() { List tableRefs = new ArrayList<>(); queryStmt.collectTableRefs(tableRefs); - if (tableRefs.size() == 1 && tableRefs.get(0) instanceof TableValuedFunctionRef - && ((TableValuedFunctionRef) tableRefs.get( - 0)).getTableFunction() instanceof GroupCommitTableValuedFunction) { - isInnerGroupCommit = true; + if (tableRefs.size() == 1 && tableRefs.get(0) instanceof TableValuedFunctionRef) { + TableValuedFunctionRef tvfRef = (TableValuedFunctionRef) tableRefs.get(0); + if (tvfRef.getTableFunction() instanceof GroupCommitTableValuedFunction) { + isGroupCommitTvf = true; + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcesslistStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcesslistStmt.java index 6e3aa5150ae5de..3b6c67b1bbaf6c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcesslistStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowProcesslistStmt.java @@ -26,14 +26,17 @@ // Used to show connection belong to this user. public class ShowProcesslistStmt extends ShowStmt { private static final ShowResultSetMetaData META_DATA = ShowResultSetMetaData.builder() + .addColumn(new Column("CurrentConnected", ScalarType.createVarchar(16))) .addColumn(new Column("Id", ScalarType.createType(PrimitiveType.BIGINT))) .addColumn(new Column("User", ScalarType.createVarchar(16))) .addColumn(new Column("Host", ScalarType.createVarchar(16))) - .addColumn(new Column("Cluster", ScalarType.createVarchar(16))) + .addColumn(new Column("LoginTime", ScalarType.createVarchar(16))) + .addColumn(new Column("Catalog", ScalarType.createVarchar(16))) .addColumn(new Column("Db", ScalarType.createVarchar(16))) .addColumn(new Column("Command", ScalarType.createVarchar(16))) .addColumn(new Column("Time", ScalarType.createType(PrimitiveType.INT))) .addColumn(new Column("State", ScalarType.createVarchar(64))) + .addColumn(new Column("QueryId", ScalarType.createVarchar(64))) .addColumn(new Column("Info", ScalarType.STRING)).build(); private boolean isFull; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateGroupSchema.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateGroupSchema.java index b5004973c37f1c..57d512b9789d9e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateGroupSchema.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateGroupSchema.java @@ -66,6 +66,10 @@ public ReplicaAllocation getReplicaAlloc() { return replicaAlloc; } + public void setReplicaAlloc(ReplicaAllocation replicaAlloc) { + this.replicaAlloc = replicaAlloc; + } + public List getDistributionColTypes() { return distributionColTypes; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java index 23703278fd890b..fcefcff132a7a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java @@ -17,10 +17,16 @@ package org.apache.doris.catalog; +import org.apache.doris.analysis.AlterColocateGroupStmt; +import org.apache.doris.clone.ColocateTableCheckerAndBalancer; +import org.apache.doris.common.DdlException; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import org.apache.doris.common.util.DynamicPartitionUtil; +import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.persist.ColocatePersistInfo; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; @@ -249,10 +255,34 @@ public void addBackendsPerBucketSeq(GroupId groupId, Map>> } } - public void addBackendsPerBucketSeqByTag(GroupId groupId, Tag tag, List> backendsPerBucketSeq) { + public void setBackendsPerBucketSeq(GroupId groupId, Map>> backendsPerBucketSeq) { writeLock(); try { + Map>> backendsPerBucketSeqMap = group2BackendsPerBucketSeq.row(groupId); + if (backendsPerBucketSeqMap != null) { + backendsPerBucketSeqMap.clear(); + } + for (Map.Entry>> entry : backendsPerBucketSeq.entrySet()) { + group2BackendsPerBucketSeq.put(groupId, entry.getKey(), entry.getValue()); + } + } finally { + writeUnlock(); + } + } + + public boolean addBackendsPerBucketSeqByTag(GroupId groupId, Tag tag, List> backendsPerBucketSeq, + ReplicaAllocation originReplicaAlloc) { + writeLock(); + try { + ColocateGroupSchema groupSchema = group2Schema.get(groupId); + // replica allocation has outdate + if (groupSchema != null && !originReplicaAlloc.equals(groupSchema.getReplicaAlloc())) { + LOG.info("replica allocation has outdate for group {}, old replica alloc {}, new replica alloc {}", + groupId, originReplicaAlloc.getAllocMap(), groupSchema.getReplicaAlloc()); + return false; + } group2BackendsPerBucketSeq.put(groupId, tag, backendsPerBucketSeq); + return true; } finally { writeUnlock(); } @@ -277,12 +307,20 @@ public void markGroupUnstable(GroupId groupId, String reason, boolean needEditLo } } - public void markGroupStable(GroupId groupId, boolean needEditLog) { + public void markGroupStable(GroupId groupId, boolean needEditLog, ReplicaAllocation originReplicaAlloc) { writeLock(); try { if (!group2Tables.containsKey(groupId)) { return; } + // replica allocation is outdate + ColocateGroupSchema groupSchema = group2Schema.get(groupId); + if (groupSchema != null && originReplicaAlloc != null + && !originReplicaAlloc.equals(groupSchema.getReplicaAlloc())) { + LOG.warn("mark group {} failed, replica alloc has outdate, old replica alloc {}, new replica alloc {}", + groupId, originReplicaAlloc.getAllocMap(), groupSchema.getReplicaAlloc()); + return; + } if (unstableGroups.remove(groupId)) { group2ErrMsgs.put(groupId, ""); if (needEditLog) { @@ -604,13 +642,23 @@ public void replayMarkGroupUnstable(ColocatePersistInfo info) { } public void replayMarkGroupStable(ColocatePersistInfo info) { - markGroupStable(info.getGroupId(), false); + markGroupStable(info.getGroupId(), false, null); } public void replayRemoveTable(ColocatePersistInfo info) { removeTable(info.getTableId()); } + public void replayModifyReplicaAlloc(ColocatePersistInfo info) throws UserException { + writeLock(); + try { + modifyColocateGroupReplicaAllocation(info.getGroupId(), info.getReplicaAlloc(), + info.getBackendsPerBucketSeq(), false); + } finally { + writeUnlock(); + } + } + // only for test public void clear() { writeLock(); @@ -633,7 +681,22 @@ public List> getInfos() { List info = Lists.newArrayList(); GroupId groupId = entry.getValue(); info.add(groupId.toString()); - info.add(entry.getKey()); + String dbName = ""; + if (groupId.dbId != 0) { + Database db = Env.getCurrentInternalCatalog().getDbNullable(groupId.dbId); + if (db != null) { + dbName = db.getFullName(); + int index = dbName.indexOf(":"); + if (index > 0) { + dbName = dbName.substring(index + 1); //use short db name + } + } + } + String groupName = entry.getKey(); + if (!GroupId.isGlobalGroupName(groupName)) { + groupName = dbName + "." + groupName.substring(groupName.indexOf("_") + 1); + } + info.add(groupName); info.add(Joiner.on(", ").join(group2Tables.get(groupId))); ColocateGroupSchema groupSchema = group2Schema.get(groupId); info.add(String.valueOf(groupSchema.getBucketsNum())); @@ -756,4 +819,124 @@ public void setErrMsgForGroup(GroupId groupId, String message) { public Map getTable2Group() { return table2Group; } + + public void alterColocateGroup(AlterColocateGroupStmt stmt) throws UserException { + writeLock(); + try { + Map properties = stmt.getProperties(); + String dbName = stmt.getColocateGroupName().getDb(); + String groupName = stmt.getColocateGroupName().getGroup(); + long dbId = 0; + if (!GroupId.isGlobalGroupName(groupName)) { + Database db = (Database) Env.getCurrentInternalCatalog().getDbOrMetaException(dbName); + dbId = db.getId(); + } + String fullGroupName = GroupId.getFullGroupName(dbId, groupName); + ColocateGroupSchema groupSchema = getGroupSchema(fullGroupName); + if (groupSchema == null) { + throw new DdlException("Not found colocate group " + stmt.getColocateGroupName().toSql()); + } + + GroupId groupId = groupSchema.getGroupId(); + + if (properties.size() > 1) { + throw new DdlException("Can only set one colocate group property at a time"); + } + + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM) + || properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) { + ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, ""); + Preconditions.checkState(!replicaAlloc.isNotSet()); + Env.getCurrentSystemInfo().checkReplicaAllocation(replicaAlloc); + Map>> backendsPerBucketSeq = getBackendsPerBucketSeq(groupId); + Map>> newBackendsPerBucketSeq = Maps.newHashMap(); + for (Map.Entry>> entry : backendsPerBucketSeq.entrySet()) { + List> newList = Lists.newArrayList(); + for (List backends : entry.getValue()) { + newList.add(Lists.newArrayList(backends)); + } + newBackendsPerBucketSeq.put(entry.getKey(), newList); + } + try { + ColocateTableCheckerAndBalancer.modifyGroupReplicaAllocation(replicaAlloc, + newBackendsPerBucketSeq, groupSchema.getBucketsNum()); + } catch (Exception e) { + LOG.warn("modify group [{}, {}] to replication allocation {} failed, bucket seq {}", + fullGroupName, groupId, replicaAlloc, backendsPerBucketSeq, e); + throw new DdlException(e.getMessage()); + } + backendsPerBucketSeq = newBackendsPerBucketSeq; + Preconditions.checkState(backendsPerBucketSeq.size() == replicaAlloc.getAllocMap().size()); + modifyColocateGroupReplicaAllocation(groupSchema.getGroupId(), replicaAlloc, + backendsPerBucketSeq, true); + } else { + throw new DdlException("Unknown colocate group property: " + properties.keySet()); + } + } finally { + writeUnlock(); + } + } + + private void modifyColocateGroupReplicaAllocation(GroupId groupId, ReplicaAllocation replicaAlloc, + Map>> backendsPerBucketSeq, boolean needEditLog) throws UserException { + ColocateGroupSchema groupSchema = getGroupSchema(groupId); + if (groupSchema == null) { + LOG.warn("not found group {}", groupId); + return; + } + + List tableIds = getAllTableIds(groupId); + for (Long tableId : tableIds) { + long dbId = groupId.dbId; + if (dbId == 0) { + dbId = groupId.getDbIdByTblId(tableId); + } + Database db = Env.getCurrentInternalCatalog().getDbNullable(dbId); + if (db == null) { + continue; + } + OlapTable table = (OlapTable) db.getTableNullable(tableId); + if (table == null || !isColocateTable(table.getId())) { + continue; + } + table.writeLock(); + try { + Map tblProperties = Maps.newHashMap(); + tblProperties.put("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, + replicaAlloc.toCreateStmt()); + table.setReplicaAllocation(tblProperties); + if (table.dynamicPartitionExists()) { + TableProperty tableProperty = table.getTableProperty(); + // Merge the new properties with origin properties, and then analyze them + Map origDynamicProperties = tableProperty.getOriginDynamicPartitionProperty(); + origDynamicProperties.put(DynamicPartitionProperty.REPLICATION_ALLOCATION, + replicaAlloc.toCreateStmt()); + Map analyzedDynamicPartition = DynamicPartitionUtil.analyzeDynamicPartition( + origDynamicProperties, table, db); + tableProperty.modifyTableProperties(analyzedDynamicPartition); + tableProperty.buildDynamicProperty(); + } + for (ReplicaAllocation alloc : table.getPartitionInfo().getPartitionReplicaAllocations().values()) { + Map allocMap = alloc.getAllocMap(); + allocMap.clear(); + allocMap.putAll(replicaAlloc.getAllocMap()); + } + } finally { + table.writeUnlock(); + } + } + + if (!backendsPerBucketSeq.equals(group2BackendsPerBucketSeq.row(groupId))) { + markGroupUnstable(groupId, "change replica allocation", false); + } + groupSchema.setReplicaAlloc(replicaAlloc); + setBackendsPerBucketSeq(groupId, backendsPerBucketSeq); + + if (needEditLog) { + ColocatePersistInfo info = ColocatePersistInfo.createForModifyReplicaAlloc(groupId, + replicaAlloc, backendsPerBucketSeq); + Env.getCurrentEnv().getEditLog().logColocateModifyRepliaAlloc(info); + } + LOG.info("modify group {} replication allocation to {}, is replay {}", groupId, replicaAlloc, !needEditLog); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java index b7ca3c622cc7b0..34f80a91038e26 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java @@ -261,6 +261,10 @@ public void setStoragePolicy(long partitionId, String storagePolicy) { idToStoragePolicy.put(partitionId, storagePolicy); } + public Map getPartitionReplicaAllocations() { + return idToReplicaAllocation; + } + public ReplicaAllocation getReplicaAllocation(long partitionId) { if (!idToReplicaAllocation.containsKey(partitionId)) { LOG.debug("failed to get replica allocation for partition: {}", partitionId); diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java index 5c18c2bd468263..4ec8993be0d2b1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java @@ -35,6 +35,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.MasterDaemon; import org.apache.doris.persist.ColocatePersistInfo; import org.apache.doris.resource.Tag; @@ -183,7 +184,12 @@ private void relocateAndBalanceGroup() { List> balancedBackendsPerBucketSeq = Lists.newArrayList(); if (relocateAndBalance(groupId, tag, unavailableBeIdsInGroup, availableBeIds, colocateIndex, infoService, statistic, balancedBackendsPerBucketSeq)) { - colocateIndex.addBackendsPerBucketSeqByTag(groupId, tag, balancedBackendsPerBucketSeq); + if (!colocateIndex.addBackendsPerBucketSeqByTag(groupId, tag, balancedBackendsPerBucketSeq, + replicaAlloc)) { + LOG.warn("relocate group {} succ, but replica allocation has change, old replica alloc {}", + groupId, replicaAlloc); + continue; + } Map>> balancedBackendsPerBucketSeqMap = Maps.newHashMap(); balancedBackendsPerBucketSeqMap.put(tag, balancedBackendsPerBucketSeq); ColocatePersistInfo info = ColocatePersistInfo @@ -219,6 +225,8 @@ private void matchGroup() { continue; } + ColocateGroupSchema groupSchema = colocateIndex.getGroupSchema(groupId); + ReplicaAllocation replicaAlloc = groupSchema.getReplicaAlloc(); String unstableReason = null; OUT: for (Long tableId : tableIds) { @@ -237,8 +245,6 @@ private void matchGroup() { olapTable.readLock(); try { for (Partition partition : olapTable.getPartitions()) { - ReplicaAllocation replicaAlloc - = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()); short replicationNum = replicaAlloc.getTotalReplicaNum(); long visibleVersion = partition.getVisibleVersion(); // Here we only get VISIBLE indexes. All other indexes are not queryable. @@ -269,8 +275,7 @@ private void matchGroup() { TabletSchedCtx tabletCtx = new TabletSchedCtx( TabletSchedCtx.Type.REPAIR, db.getId(), tableId, partition.getId(), index.getId(), tablet.getId(), - olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()), - System.currentTimeMillis()); + replicaAlloc, System.currentTimeMillis()); // the tablet status will be set again when being scheduled tabletCtx.setTabletStatus(st); tabletCtx.setPriority(Priority.NORMAL); @@ -299,7 +304,7 @@ private void matchGroup() { // mark group as stable or unstable if (Strings.isNullOrEmpty(unstableReason)) { - colocateIndex.markGroupStable(groupId, true); + colocateIndex.markGroupStable(groupId, true, replicaAlloc); } else { colocateIndex.markGroupUnstable(groupId, unstableReason, true); } @@ -521,6 +526,122 @@ private List> getHostsPerBucketSeq(List> backendsPerBuck return hostsPerBucketSeq; } + public static void modifyGroupReplicaAllocation(ReplicaAllocation replicaAlloc, + Map>> backendBucketsSeq, int bucketNum) throws Exception { + Map allocMap = replicaAlloc.getAllocMap(); + List deleteTags = Lists.newArrayList(); + for (Tag tag : backendBucketsSeq.keySet()) { + if (!allocMap.containsKey(tag)) { + deleteTags.add(tag); + } + Preconditions.checkState(bucketNum == backendBucketsSeq.get(tag).size(), + bucketNum + " vs " + backendBucketsSeq.get(tag).size()); + } + deleteTags.forEach(tag -> backendBucketsSeq.remove(tag)); + + for (Tag tag : replicaAlloc.getAllocMap().keySet()) { + if (!backendBucketsSeq.containsKey(tag)) { + List> tagBackendBucketsSeq = Lists.newArrayList(); + for (int i = 0; i < bucketNum; i++) { + tagBackendBucketsSeq.add(Lists.newArrayList()); + } + backendBucketsSeq.put(tag, tagBackendBucketsSeq); + } + } + + Map backendToBucketNum = Maps.newHashMap(); + backendBucketsSeq.values().forEach(tagBackendIds -> + tagBackendIds.forEach(backendIds -> + backendIds.forEach(backendId -> backendToBucketNum.put( + backendId, backendToBucketNum.getOrDefault(backendId, 0) + 1)))); + + for (Tag tag : backendBucketsSeq.keySet()) { + List> tagBackendBucketsSeq = backendBucketsSeq.get(tag); + int oldReplicaNum = tagBackendBucketsSeq.get(0).size(); + for (List backendIdsOneBucket : tagBackendBucketsSeq) { + Preconditions.checkState(backendIdsOneBucket.size() == oldReplicaNum, + backendIdsOneBucket.size() + " vs " + oldReplicaNum); + } + + int newReplicaNum = allocMap.get(tag); + if (newReplicaNum == oldReplicaNum) { + continue; + } + + List backends = Env.getCurrentSystemInfo().getBackendsByTag(tag); + Set availableBeIds = backends.stream().filter(be -> be.isScheduleAvailable()) + .map(be -> be.getId()).collect(Collectors.toSet()); + + for (Long backendId : availableBeIds) { + if (!backendToBucketNum.containsKey(backendId)) { + backendToBucketNum.put(backendId, 0); + } + } + + for (int i = 0; i < tagBackendBucketsSeq.size(); i++) { + modifyGroupBucketReplicas(tag, newReplicaNum, tagBackendBucketsSeq.get(i), + availableBeIds, backendToBucketNum); + } + } + } + + private static void modifyGroupBucketReplicas(Tag tag, int newReplicaNum, List backendIds, + Set availableBeIds, Map backendToBucketNum) throws Exception { + final boolean smallIdFirst = Math.random() < 0.5; + if (backendIds.size() > newReplicaNum) { + backendIds.sort((id1, id2) -> { + boolean alive1 = availableBeIds.contains(id1); + boolean alive2 = availableBeIds.contains(id2); + if (alive1 != alive2) { + return alive1 ? -1 : 1; + } + int bucketNum1 = backendToBucketNum.getOrDefault(id1, 0); + int bucketNum2 = backendToBucketNum.getOrDefault(id2, 0); + if (bucketNum1 != bucketNum2) { + return Integer.compare(bucketNum1, bucketNum2); + } + + return smallIdFirst ? Long.compare(id1, id2) : Long.compare(id2, id1); + }); + + for (int i = backendIds.size() - 1; i >= newReplicaNum; i--) { + long backendId = backendIds.get(i); + backendIds.remove(i); + backendToBucketNum.put(backendId, backendToBucketNum.getOrDefault(backendId, 0) - 1); + } + } + + if (backendIds.size() < newReplicaNum) { + Set candBackendSet = Sets.newHashSet(); + candBackendSet.addAll(availableBeIds); + candBackendSet.removeAll(backendIds); + if (backendIds.size() + candBackendSet.size() < newReplicaNum) { + throw new UserException("Can not add backend for tag: " + tag); + } + + List candBackendList = Lists.newArrayList(candBackendSet); + candBackendList.sort((id1, id2) -> { + int bucketNum1 = backendToBucketNum.getOrDefault(id1, 0); + int bucketNum2 = backendToBucketNum.getOrDefault(id2, 0); + if (bucketNum1 != bucketNum2) { + return Integer.compare(bucketNum1, bucketNum2); + } + + return smallIdFirst ? Long.compare(id1, id2) : Long.compare(id2, id1); + }); + + int addNum = newReplicaNum - backendIds.size(); + for (int i = 0; i < addNum; i++) { + long backendId = candBackendList.get(i); + backendIds.add(backendId); + backendToBucketNum.put(backendId, backendToBucketNum.getOrDefault(backendId, 0) + 1); + } + } + + Preconditions.checkState(newReplicaNum == backendIds.size(), + newReplicaNum + " vs " + backendIds.size()); + } + private List> getSortedBackendReplicaNumPairs(List allAvailBackendIds, Set unavailBackendIds, LoadStatisticForTag statistic, List flatBackendsPerBucketSeq) { // backend id -> replica num, and sorted by replica num, descending. diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java index 53f44070d93957..dc03ecf82330a3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java @@ -221,7 +221,7 @@ private static int getBucketsNum(DynamicPartitionProperty property, OlapTable ta // plus 5 for uncompressed data long uncompressedPartitionSize = getNextPartitionSize(partitionSizeArray) * 5; - return AutoBucketUtils.getBucketsNum(uncompressedPartitionSize); + return AutoBucketUtils.getBucketsNum(uncompressedPartitionSize, Config.autobucket_min_buckets); } private ArrayList getAddPartitionClause(Database db, OlapTable olapTable, diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java index d6a8e1efa0e8f6..ee9da3ac10008f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.AdminCancelRebalanceDiskStmt; import org.apache.doris.analysis.AdminRebalanceDiskStmt; +import org.apache.doris.catalog.ColocateGroupSchema; import org.apache.doris.catalog.ColocateTableIndex; import org.apache.doris.catalog.ColocateTableIndex.GroupId; import org.apache.doris.catalog.Database; @@ -490,15 +491,20 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) throw new SchedException(Status.UNRECOVERABLE, "index does not exist"); } + ReplicaAllocation replicaAlloc = null; Tablet tablet = idx.getTablet(tabletId); Preconditions.checkNotNull(tablet); - ReplicaAllocation replicaAlloc = tbl.getPartitionInfo().getReplicaAllocation(partition.getId()); - if (isColocateTable) { GroupId groupId = colocateTableIndex.getGroup(tbl.getId()); if (groupId == null) { throw new SchedException(Status.UNRECOVERABLE, "colocate group does not exist"); } + ColocateGroupSchema groupSchema = colocateTableIndex.getGroupSchema(groupId); + if (groupSchema == null) { + throw new SchedException(Status.UNRECOVERABLE, + "colocate group schema " + groupId + " does not exist"); + } + replicaAlloc = groupSchema.getReplicaAlloc(); int tabletOrderIdx = tabletCtx.getTabletOrderIdx(); if (tabletOrderIdx == -1) { @@ -512,6 +518,7 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) statusPair = Pair.of(st, Priority.HIGH); tabletCtx.setColocateGroupBackendIds(backendsSet); } else { + replicaAlloc = tbl.getPartitionInfo().getReplicaAllocation(partition.getId()); List aliveBeIds = infoService.getAllBackendIds(true); statusPair = tablet.getHealthStatusWithPriority( infoService, partition.getVisibleVersion(), replicaAlloc, aliveBeIds); @@ -1484,14 +1491,18 @@ private void tryAddAfterFinished(TabletSchedCtx tabletCtx) { return; } - replicaAlloc = tbl.getPartitionInfo().getReplicaAllocation(partition.getId()); boolean isColocateTable = colocateTableIndex.isColocateTable(tbl.getId()); if (isColocateTable) { GroupId groupId = colocateTableIndex.getGroup(tbl.getId()); if (groupId == null) { return; } + ColocateGroupSchema groupSchema = colocateTableIndex.getGroupSchema(groupId); + if (groupSchema == null) { + return; + } + replicaAlloc = groupSchema.getReplicaAlloc(); int tabletOrderIdx = tabletCtx.getTabletOrderIdx(); if (tabletOrderIdx == -1) { tabletOrderIdx = idx.getTabletOrderIdx(tablet.getId()); @@ -1504,6 +1515,7 @@ private void tryAddAfterFinished(TabletSchedCtx tabletCtx) { statusPair = Pair.of(st, Priority.HIGH); tabletCtx.setColocateGroupBackendIds(backendsSet); } else { + replicaAlloc = tbl.getPartitionInfo().getReplicaAllocation(partition.getId()); List aliveBeIds = infoService.getAllBackendIds(true); statusPair = tablet.getHealthStatusWithPriority( infoService, partition.getVisibleVersion(), replicaAlloc, aliveBeIds); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletHealthProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletHealthProcDir.java index 93f54483cbfa5e..3ce3ff74c7adaf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletHealthProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletHealthProcDir.java @@ -17,6 +17,7 @@ package org.apache.doris.common.proc; +import org.apache.doris.catalog.ColocateGroupSchema; import org.apache.doris.catalog.ColocateTableIndex; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; @@ -185,6 +186,10 @@ static class DBTabletStatistic { ++tabletNum; Tablet.TabletStatus res = null; if (groupId != null) { + ColocateGroupSchema groupSchema = colocateTableIndex.getGroupSchema(groupId); + if (groupSchema != null) { + replicaAlloc = groupSchema.getReplicaAlloc(); + } Set backendsSet = colocateTableIndex.getTabletBackendsByGroup(groupId, i); res = tablet.getColocateHealthStatus(partition.getVisibleVersion(), replicaAlloc, backendsSet); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/AutoBucketUtils.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/AutoBucketUtils.java index ca935ab20e7b9b..294250fd213bc6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/AutoBucketUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/AutoBucketUtils.java @@ -95,4 +95,9 @@ public static int getBucketsNum(long partitionSize) { logger.debug("AutoBucketsUtil: final bucketsNum {}", bucketsNum); return bucketsNum; } + + public static int getBucketsNum(long partitionSize, int minBuckets) { + int bucketsNum = getBucketsNum(partitionSize); + return Math.max(minBuckets, bucketsNum); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/Util.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/Util.java index ae471ce8dd9019..01198f9493b083 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/Util.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/Util.java @@ -563,6 +563,8 @@ public static TFileFormatType getFileFormatTypeFromName(String formatName) { // TODO: Add TEXTFILE to TFileFormatType to Support hive text file format. || lowerFileFormat.equals(FeConstants.text)) { return TFileFormatType.FORMAT_CSV_PLAIN; + } else if (lowerFileFormat.equals("wal")) { + return TFileFormatType.FORMAT_WAL; } else { return TFileFormatType.FORMAT_UNKNOWN; } 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 bc752179b0d9cf..057f18d2fc0809 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 @@ -266,7 +266,8 @@ private String execute(String path) throws DorisEsException { if (response.isSuccessful()) { return response.body().string(); } else { - LOG.warn("request response code: {}, body: {}", response.code(), response.body().string()); + LOG.warn("request response code: {}, body: {}", response.code(), response.message()); + scratchExceptionForThrow = new DorisEsException(response.message()); } } catch (IOException e) { LOG.warn("request node [{}] [{}] failures {}, try next nodes", currentNode, path, e); diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/SessionController.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/SessionController.java index 53a32a579697bd..27a28dbb4f412a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/SessionController.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/SessionController.java @@ -58,14 +58,17 @@ public class SessionController extends RestBaseController { private static final Logger LOG = LogManager.getLogger(SessionController.class); static { + SESSION_TABLE_HEADER.add("CurrentConnected"); SESSION_TABLE_HEADER.add("Id"); SESSION_TABLE_HEADER.add("User"); SESSION_TABLE_HEADER.add("Host"); - SESSION_TABLE_HEADER.add("Cluster"); + SESSION_TABLE_HEADER.add("LoginTime"); + SESSION_TABLE_HEADER.add("Catalog"); SESSION_TABLE_HEADER.add("Db"); SESSION_TABLE_HEADER.add("Command"); SESSION_TABLE_HEADER.add("Time"); SESSION_TABLE_HEADER.add("State"); + SESSION_TABLE_HEADER.add("QueryId"); SESSION_TABLE_HEADER.add("Info"); ALL_SESSION_TABLE_HEADER.addAll(SESSION_TABLE_HEADER); } @@ -111,7 +114,7 @@ private List> getSessionInfo(boolean showFe) { .listConnection("root", false); long nowMs = System.currentTimeMillis(); return threadInfos.stream() - .map(info -> info.toRow(nowMs, showFe)) + .map(info -> info.toRow(-1, nowMs, showFe)) .map(row -> { Map record = new HashMap<>(); for (int i = 0; i < row.size(); i++) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/meta/ColocateMetaService.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/meta/ColocateMetaService.java index 9e51d38de5cfcc..b7c2a615aac280 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/meta/ColocateMetaService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/meta/ColocateMetaService.java @@ -114,7 +114,7 @@ public Object group_stable(HttpServletRequest request, HttpServletResponse respo if ("POST".equalsIgnoreCase(method)) { colocateIndex.markGroupUnstable(groupId, "mark unstable via http api", true); } else if ("DELETE".equalsIgnoreCase(method)) { - colocateIndex.markGroupStable(groupId, true); + colocateIndex.markGroupStable(groupId, true, null); } return ResponseEntityBuilder.ok(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index ce2768b46eac29..80ee1a19aac5f4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -469,6 +469,7 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } + case OperationType.OP_COLOCATE_MOD_REPLICA_ALLOC: case OperationType.OP_COLOCATE_ADD_TABLE: case OperationType.OP_COLOCATE_REMOVE_TABLE: case OperationType.OP_COLOCATE_BACKENDS_PER_BUCKETSEQ: diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index 2e6be68604160a..4461ba19473a02 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.BinlogConfig; +import org.apache.doris.catalog.ColocateGroupSchema; import org.apache.doris.catalog.ColocateTableIndex; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; @@ -1172,6 +1173,10 @@ private static boolean addReplica(long tabletId, TabletMeta tabletMeta, TTabletI int tabletOrderIdx = materializedIndex.getTabletOrderIdx(tabletId); Preconditions.checkState(tabletOrderIdx != -1, "get tablet materializedIndex for %s fail", tabletId); Set backendsSet = colocateTableIndex.getTabletBackendsByGroup(groupId, tabletOrderIdx); + ColocateGroupSchema groupSchema = colocateTableIndex.getGroupSchema(groupId); + if (groupSchema != null) { + replicaAlloc = groupSchema.getReplicaAlloc(); + } TabletStatus status = tablet.getColocateHealthStatus(visibleVersion, replicaAlloc, backendsSet); if (status == TabletStatus.HEALTHY) { 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 b80b622190dbc1..5bbb7b26973a4c 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 @@ -358,7 +358,7 @@ public PlanFragment visitPhysicalOlapTableSink(PhysicalOlapTableSink partialUpdateCols = new HashSet<>(); boolean isPartialUpdate = olapTableSink.isPartialUpdate(); if (isPartialUpdate) { - OlapTable olapTable = (OlapTable) olapTableSink.getTargetTable(); + OlapTable olapTable = olapTableSink.getTargetTable(); if (!olapTable.getEnableUniqueKeyMergeOnWrite()) { throw new AnalysisException("Partial update is only allowed in" + "unique table with merge-on-write enabled."); @@ -395,6 +395,7 @@ public PlanFragment visitPhysicalOlapTableSink(PhysicalOlapTableSink buildRules() { return ImmutableList.of( - RuleType.BINDING_INSERT_TARGET_TABLE.build( - unboundOlapTableSink().thenApply(ctx -> { - UnboundOlapTableSink sink = ctx.root; - Pair pair = bind(ctx.cascadesContext, sink); - Database database = pair.first; - OlapTable table = pair.second; + RuleType.BINDING_INSERT_TARGET_TABLE.build(unboundOlapTableSink().thenApply(ctx -> { + UnboundOlapTableSink sink = ctx.root; + Pair pair = bind(ctx.cascadesContext, sink); + Database database = pair.first; + OlapTable table = pair.second; - LogicalPlan child = ((LogicalPlan) sink.child()); + LogicalPlan child = ((LogicalPlan) sink.child()); + boolean isNeedSequenceCol = child.getOutput().stream() + .anyMatch(slot -> slot.getName().equals(Column.SEQUENCE_COL)); - LogicalOlapTableSink boundSink = new LogicalOlapTableSink<>( - database, - table, - bindTargetColumns(table, sink.getColNames()), - bindPartitionIds(table, sink.getPartitions()), - child.getOutput().stream() - .map(NamedExpression.class::cast) - .collect(ImmutableList.toImmutableList()), - sink.isPartialUpdate(), - sink.isFromNativeInsertStmt(), - sink.child()); + LogicalOlapTableSink boundSink = new LogicalOlapTableSink<>( + database, + table, + bindTargetColumns(table, sink.getColNames(), isNeedSequenceCol), + bindPartitionIds(table, sink.getPartitions()), + child.getOutput().stream() + .map(NamedExpression.class::cast) + .collect(ImmutableList.toImmutableList()), + sink.isPartialUpdate(), + sink.isFromNativeInsertStmt(), + sink.child()); - // we need to insert all the columns of the target table - // although some columns are not mentions. - // so we add a projects to supply the default value. + // we need to insert all the columns of the target table + // although some columns are not mentions. + // so we add a projects to supply the default value. - if (boundSink.getCols().size() != child.getOutput().size()) { - throw new AnalysisException( - "insert into cols should be corresponding to the query output"); - } + if (boundSink.getCols().size() != child.getOutput().size()) { + throw new AnalysisException("insert into cols should be corresponding to the query output"); + } - Map columnToChildOutput = Maps.newHashMap(); - for (int i = 0; i < boundSink.getCols().size(); ++i) { - columnToChildOutput.put(boundSink.getCols().get(i), child.getOutput().get(i)); - } + Map columnToChildOutput = Maps.newHashMap(); + for (int i = 0; i < boundSink.getCols().size(); ++i) { + columnToChildOutput.put(boundSink.getCols().get(i), child.getOutput().get(i)); + } - Map columnToOutput = Maps.newLinkedHashMap(); - NereidsParser expressionParser = new NereidsParser(); + Map columnToOutput = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + NereidsParser expressionParser = new NereidsParser(); - // this is a trick way to avoid legacy planner's slotRef toSql output include label. - // see more in org.apache.doris.analysis.SlotRef.toSqlImpl - if (ConnectContext.get() != null) { - ConnectContext.get().getState().setIsQuery(true); + // generate slots not mentioned in sql, mv slots and shaded slots. + for (Column column : boundSink.getTargetTable().getFullSchema()) { + if (column.isMaterializedViewColumn()) { + List refs = column.getRefColumns(); + // now we have to replace the column to slots. + Preconditions.checkArgument(refs != null, + "mv column %s 's ref column cannot be null", column); + Expression parsedExpression = expressionParser.parseExpression( + column.getDefineExpr().toSqlWithoutTbl()); + Expression boundSlotExpression = SlotReplacer.INSTANCE + .replace(parsedExpression, columnToOutput); + // the boundSlotExpression is an expression whose slots are bound but function + // may not be bound, we have to bind it again. + // for example: to_bitmap. + Expression boundExpression = FunctionBinder.INSTANCE.rewrite( + boundSlotExpression, new ExpressionRewriteContext(ctx.cascadesContext)); + if (boundExpression instanceof Alias) { + boundExpression = ((Alias) boundExpression).child(); } - try { - // generate slots not mentioned in sql, mv slots and shaded slots. - for (Column column : boundSink.getTargetTable().getFullSchema()) { - if (column.isMaterializedViewColumn()) { - List refs = column.getRefColumns(); - // now we have to replace the column to slots. - Preconditions.checkArgument(refs != null, - "mv column's ref column cannot be null"); - Expression parsedExpression = expressionParser.parseExpression( - column.getDefineExpr().toSql()); - Expression boundSlotExpression = SlotReplacer.INSTANCE - .replace(parsedExpression, columnToOutput); - // the boundSlotExpression is an expression whose slots are bound but function - // may not be bound, we have to bind it again. - // for example: to_bitmap. - Expression boundExpression = FunctionBinder.INSTANCE.rewrite( - boundSlotExpression, new ExpressionRewriteContext(ctx.cascadesContext)); - - NamedExpression slot = boundExpression instanceof NamedExpression - ? ((NamedExpression) boundExpression) - : new Alias(boundExpression); - - columnToOutput.put(column.getName(), slot); - } else if (columnToChildOutput.containsKey(column)) { - columnToOutput.put(column.getName(), columnToChildOutput.get(column)); + NamedExpression slot = new Alias(boundExpression, column.getDefineExpr().toSqlWithoutTbl()); + columnToOutput.put(column.getName(), slot); + } else if (columnToChildOutput.containsKey(column)) { + columnToOutput.put(column.getName(), columnToChildOutput.get(column)); + } else { + if (table.hasSequenceCol() + && column.getName().equals(Column.SEQUENCE_COL) + && table.getSequenceMapCol() != null) { + Optional seqCol = table.getFullSchema().stream() + .filter(col -> col.getName().equals(table.getSequenceMapCol())) + .findFirst(); + if (!seqCol.isPresent()) { + throw new AnalysisException("sequence column is not contained in" + + " target table " + table.getName()); + } + columnToOutput.put(column.getName(), columnToOutput.get(seqCol.get().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() + )); + } else { + try { + // it comes from the original planner, if default value expression is + // null, we use the literal string of the default value, or it may be + // default value function, like CURRENT_TIMESTAMP. + if (column.getDefaultValueExpr() == null) { + columnToOutput.put(column.getName(), + new Alias(Literal.of(column.getDefaultValue()) + .checkedCastTo(DataType.fromCatalogType(column.getType())), + column.getName())); } else { - if (table.hasSequenceCol() - && column.getName().equals(Column.SEQUENCE_COL) - && table.getSequenceMapCol() != null) { - Column seqCol = table.getFullSchema().stream() - .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() - )); - } else { - try { - // it comes from the original planner, if default value expression is - // null, we use the literal string of the default value, or it may be - // default value function, like CURRENT_TIMESTAMP. - if (column.getDefaultValueExpr() == null) { - columnToOutput.put(column.getName(), - new Alias(Literal.of(column.getDefaultValue()) - .checkedCastTo( - DataType.fromCatalogType(column.getType())), - column.getName())); - } else { - Expression defualtValueExpression = FunctionBinder.INSTANCE.rewrite( - new NereidsParser().parseExpression( - column.getDefaultValueExpr().toSql()), - new ExpressionRewriteContext(ctx.cascadesContext)); - NamedExpression slot = - defualtValueExpression instanceof NamedExpression - ? ((NamedExpression) defualtValueExpression) - : new Alias(defualtValueExpression); - - columnToOutput.put(column.getName(), slot); - } - } catch (Exception e) { - throw new AnalysisException(e.getMessage(), e.getCause()); - } + Expression defualtValueExpression = FunctionBinder.INSTANCE.rewrite( + new NereidsParser().parseExpression( + column.getDefaultValueExpr().toSqlWithoutTbl()), + new ExpressionRewriteContext(ctx.cascadesContext)); + if (defualtValueExpression instanceof Alias) { + defualtValueExpression = ((Alias) defualtValueExpression).child(); } + columnToOutput.put(column.getName(), + new Alias(defualtValueExpression, column.getName())); } - } - } finally { - if (ConnectContext.get() != null) { - // this is a trick way to avoid legacy planner's slotRef toSql output include label - // set back to original value. - ConnectContext.get().getState().setIsQuery(false); + } catch (Exception e) { + throw new AnalysisException(e.getMessage(), e.getCause()); } } - List fullOutputExprs = ImmutableList.copyOf(columnToOutput.values()); + } + } + List fullOutputExprs = ImmutableList.copyOf(columnToOutput.values()); - LogicalProject fullOutputProject = new LogicalProject<>(fullOutputExprs, - boundSink.child()); + LogicalProject fullOutputProject = new LogicalProject<>(fullOutputExprs, boundSink.child()); - // add cast project - List castExprs = Lists.newArrayList(); - for (int i = 0; i < table.getFullSchema().size(); ++i) { - 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 { - castExprs.add(new Alias(castExpr)); - } - } - if (!castExprs.equals(fullOutputExprs)) { - fullOutputProject = new LogicalProject(castExprs, fullOutputProject); + // add cast project + List castExprs = Lists.newArrayList(); + for (int i = 0; i < table.getFullSchema().size(); ++i) { + Column col = table.getFullSchema().get(i); + NamedExpression expr = 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; + } + maybeFallbackCastUnsupportedType(expr, ctx.connectContext); + DataType inputType = expr.getDataType(); + DataType targetType = DataType.fromCatalogType(table.getFullSchema().get(i).getType()); + Expression castExpr = expr; + if (isSourceAndTargetStringLikeType(inputType, targetType)) { + int sourceLength = ((CharacterType) inputType).getLen(); + int targetLength = ((CharacterType) targetType).getLen(); + if (sourceLength >= targetLength && targetLength >= 0) { + castExpr = new Substring(castExpr, Literal.of(1), Literal.of(targetLength)); + } else if (targetType.isStringType()) { + castExpr = new Cast(castExpr, StringType.INSTANCE); } + } else { + castExpr = TypeCoercionUtils.castIfNotSameType(castExpr, targetType); + } + if (castExpr instanceof NamedExpression) { + castExprs.add(((NamedExpression) castExpr)); + } else { + castExprs.add(new Alias(castExpr)); + } + } + if (!castExprs.equals(fullOutputExprs)) { + fullOutputProject = new LogicalProject(castExprs, fullOutputProject); + } - return boundSink.withChildAndUpdateOutput(fullOutputProject); + return boundSink.withChildAndUpdateOutput(fullOutputProject); - })), + })), RuleType.BINDING_INSERT_FILE.build( logicalFileSink().when(s -> s.getOutputExprs().isEmpty()) .then(fileSink -> fileSink.withOutputExprs( @@ -238,6 +242,11 @@ private Pair bind(CascadesContext cascadesContext, UnboundO Pair pair = RelationUtil.getDbAndTable(tableQualifier, cascadesContext.getConnectContext().getEnv()); if (!(pair.second instanceof OlapTable)) { + try { + cascadesContext.getConnectContext().getSessionVariable().enableFallbackToOriginalPlannerOnce(); + } catch (Exception e) { + throw new AnalysisException("fall back failed"); + } throw new AnalysisException("the target table of insert into is not an OLAP table"); } return Pair.of(((Database) pair.first), (OlapTable) pair.second); @@ -256,11 +265,12 @@ private List bindPartitionIds(OlapTable table, List partitions) { }).collect(Collectors.toList()); } - private List bindTargetColumns(OlapTable table, List colsName) { + private List bindTargetColumns(OlapTable table, List colsName, boolean isNeedSequenceCol) { + // if the table set sequence column in stream load phase, the sequence map column is null, we query it. return colsName.isEmpty() - ? table.getFullSchema().stream().filter(column -> column.isVisible() - && !column.isMaterializedViewColumn()) - .collect(Collectors.toList()) + ? table.getFullSchema().stream() + .filter(c -> validColumn(c, isNeedSequenceCol)) + .collect(ImmutableList.toImmutableList()) : colsName.stream().map(cn -> { Column column = table.getColumn(cn); if (column == null) { @@ -268,7 +278,27 @@ private List bindTargetColumns(OlapTable table, List colsName) { cn, table.getName())); } return column; - }).collect(Collectors.toList()); + }).collect(ImmutableList.toImmutableList()); + } + + private void maybeFallbackCastUnsupportedType(Expression expression, ConnectContext ctx) { + if (expression.getDataType().isMapType()) { + try { + ctx.getSessionVariable().enableFallbackToOriginalPlannerOnce(); + } catch (Exception e) { + throw new AnalysisException("failed to try to fall back to original planner"); + } + throw new AnalysisException("failed to cast type when binding sink, type is: " + expression.getDataType()); + } + } + + private boolean isSourceAndTargetStringLikeType(DataType input, DataType target) { + return input.isStringLikeType() && target.isStringLikeType(); + } + + private boolean validColumn(Column column, boolean isNeedSequenceCol) { + return (column.isVisible() || (isNeedSequenceCol && column.isSequenceColumn())) + && !column.isMaterializedViewColumn(); } private static class SlotReplacer extends DefaultExpressionRewriter> { @@ -280,6 +310,9 @@ public Expression replace(Expression e, Map replaceMap) @Override public Expression visitUnboundSlot(UnboundSlot unboundSlot, Map replaceMap) { + if (!replaceMap.containsKey(unboundSlot.getName())) { + throw new AnalysisException("cannot find column from target table " + unboundSlot.getNameParts()); + } return replaceMap.get(unboundSlot.getName()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java index de67fea9355713..7ea406677edde1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.BinaryOperator; +import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; import org.apache.doris.nereids.trees.expressions.Exists; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.InSubquery; @@ -30,6 +31,7 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.ScalarSubquery; +import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.SubqueryExpr; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; @@ -38,6 +40,7 @@ import org.apache.doris.nereids.trees.plans.algebra.Aggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalApply; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; @@ -52,6 +55,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; /** * SubqueryToApply. translate from subquery to LogicalApply. @@ -68,7 +72,7 @@ public List buildRules() { LogicalFilter filter = ctx.root; ImmutableList> subqueryExprsList = filter.getConjuncts().stream() - .map(e -> (Set) e.collect(SubqueryExpr.class::isInstance)) + .>map(e -> e.collect(SubqueryExpr.class::isInstance)) .collect(ImmutableList.toImmutableList()); if (subqueryExprsList.stream() .flatMap(Collection::stream).noneMatch(SubqueryExpr.class::isInstance)) { @@ -117,7 +121,7 @@ public List buildRules() { RuleType.PROJECT_SUBQUERY_TO_APPLY.build(logicalProject().thenApply(ctx -> { LogicalProject project = ctx.root; ImmutableList> subqueryExprsList = project.getProjects().stream() - .map(e -> (Set) e.collect(SubqueryExpr.class::isInstance)) + .>map(e -> e.collect(SubqueryExpr.class::isInstance)) .collect(ImmutableList.toImmutableList()); if (subqueryExprsList.stream().flatMap(Collection::stream).count() == 0) { return project; @@ -164,11 +168,84 @@ public List buildRules() { oneRowRelation.withProjects( ImmutableList.of(new Alias(BooleanLiteral.of(true), ctx.statementContext.generateColumnName())))); - } - )) + })), + RuleType.JOIN_SUBQUERY_TO_APPLY + .build(logicalJoin() + .when(join -> join.getHashJoinConjuncts().isEmpty() && !join.getOtherJoinConjuncts().isEmpty()) + .thenApply(ctx -> { + LogicalJoin join = ctx.root; + Map> joinConjuncts = join.getOtherJoinConjuncts().stream() + .collect(Collectors.groupingBy(conjunct -> conjunct.containsType(SubqueryExpr.class), + Collectors.toList())); + List subqueryConjuncts = joinConjuncts.get(true); + if (subqueryConjuncts == null || subqueryConjuncts.stream() + .anyMatch(expr -> !isValidSubqueryConjunct(expr, join.left()))) { + return join; + } + + ImmutableList> subqueryExprsList = subqueryConjuncts.stream() + .>map(e -> e.collect(SubqueryExpr.class::isInstance)) + .collect(ImmutableList.toImmutableList()); + ImmutableList.Builder newConjuncts = new ImmutableList.Builder<>(); + LogicalPlan applyPlan = null; + LogicalPlan leftChildPlan = (LogicalPlan) join.left(); + + // Subquery traversal with the conjunct of and as the granularity. + for (int i = 0; i < subqueryExprsList.size(); ++i) { + Set subqueryExprs = subqueryExprsList.get(i); + if (subqueryExprs.size() > 1) { + // only support the conjunct contains one subquery expr + return join; + } + + // first step: Replace the subquery of predicate in LogicalFilter + // second step: Replace subquery with LogicalApply + ReplaceSubquery replaceSubquery = new ReplaceSubquery(ctx.statementContext, true); + SubqueryContext context = new SubqueryContext(subqueryExprs); + Expression conjunct = replaceSubquery.replace(subqueryConjuncts.get(i), context); + + applyPlan = subqueryToApply( + subqueryExprs.stream().collect(ImmutableList.toImmutableList()), + leftChildPlan, context.getSubqueryToMarkJoinSlot(), + ctx.cascadesContext, Optional.of(conjunct), false); + leftChildPlan = applyPlan; + newConjuncts.add(conjunct); + } + List simpleConjuncts = joinConjuncts.get(false); + if (simpleConjuncts != null) { + newConjuncts.addAll(simpleConjuncts); + } + Plan newJoin = join.withConjunctsChildren(join.getHashJoinConjuncts(), + newConjuncts.build(), applyPlan, join.right()); + return newJoin; + })) ); } + private static boolean isValidSubqueryConjunct(Expression expression, Plan leftChild) { + // the subquery must be uncorrelated subquery or only correlated to the left child + // currently only support the following 4 simple scenarios + // 1. col ComparisonPredicate subquery + // 2. col in (subquery) + // 3. exists (subquery) + // 4. col1 ComparisonPredicate subquery or xxx (no more subquery) + List slots = leftChild.getOutput(); + if (expression instanceof ComparisonPredicate && expression.child(1) instanceof ScalarSubquery) { + ScalarSubquery subquery = (ScalarSubquery) expression.child(1); + return slots.containsAll(subquery.getCorrelateSlots()); + } else if (expression instanceof InSubquery) { + return slots.containsAll(((InSubquery) expression).getCorrelateSlots()); + } else if (expression instanceof Exists) { + return slots.containsAll(((Exists) expression).getCorrelateSlots()); + } else { + List subqueryExprs = expression.collectToList(SubqueryExpr.class::isInstance); + if (subqueryExprs.size() == 1) { + return slots.containsAll(subqueryExprs.get(0).getCorrelateSlots()); + } + } + return false; + } + private LogicalPlan subqueryToApply(List subqueryExprs, LogicalPlan childPlan, Map> subqueryToMarkJoinSlot, CascadesContext ctx, @@ -252,12 +329,12 @@ private static class ReplaceSubquery extends DefaultExpressionRewriterDateTimeLiteral - | | - | +----->DateTimeV2Literal - +--->DateV2Literal - */ - private Expression migrateLiteralToDateTime(DateLiteral l) { - if (l instanceof DateV2Literal) { - return new DateTimeLiteral(l.getYear(), l.getMonth(), l.getDay(), 0, 0, 0); - } else if (l instanceof DateTimeV2Literal) { - DateTimeV2Literal dtv2 = (DateTimeV2Literal) l; - return new DateTimeLiteral(dtv2.getYear(), dtv2.getMonth(), dtv2.getDay(), - dtv2.getHour(), dtv2.getMinute(), dtv2.getSecond()); - } else if (l instanceof DateTimeLiteral) { - return l; - } else if (l instanceof DateLiteral) { - return new DateTimeLiteral(l.getYear(), l.getMonth(), l.getDay(), 0, 0, 0); - } - throw new AnalysisException("cannot convert" + l.toSql() + " to DateTime"); - } - private Expression migrateToDateTime(DateTimeV2Literal l) { return new DateTimeLiteral(l.getYear(), l.getMonth(), l.getDay(), l.getHour(), l.getMinute(), l.getSecond()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java index a2233ed9c55b68..00f55936c1834f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.expressions.GreaterThan; import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; import org.apache.doris.nereids.trees.expressions.InPredicate; +import org.apache.doris.nereids.trees.expressions.IsNull; import org.apache.doris.nereids.trees.expressions.LessThan; import org.apache.doris.nereids.trees.expressions.LessThanEqual; import org.apache.doris.nereids.trees.expressions.Like; @@ -386,6 +387,22 @@ public Statistics visitNot(Not not, EstimationContext context) { return statisticsBuilder.build(); } + @Override + public Statistics visitIsNull(IsNull isNull, EstimationContext context) { + ColumnStatistic childStats = ExpressionEstimation.estimate(isNull.child(), context.statistics); + if (childStats.isUnKnown()) { + return new StatisticsBuilder(context.statistics).build(); + } + double outputRowCount = childStats.numNulls; + ColumnStatisticBuilder colBuilder = new ColumnStatisticBuilder(childStats); + // do not modify ndv/min/max to make is-not-null work + colBuilder.setCount(outputRowCount).setNumNulls(outputRowCount); + StatisticsBuilder builder = new StatisticsBuilder(context.statistics); + builder.putColumnStatistics(isNull.child(), colBuilder.build()); + // TODO we do not call updateRowCountOnly() to make is-not-null work. this need refactor + return builder.build(); + } + static class EstimationContext { private final Statistics statistics; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/InPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/InPredicate.java index a7383d55101401..d839a1e9062b0b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/InPredicate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/InPredicate.java @@ -90,7 +90,7 @@ public String toString() { @Override public String toSql() { return compareExpr.toSql() + " IN " + options.stream() - .map(Expression::toSql) + .map(Expression::toSql).sorted() .collect(Collectors.joining(", ", "(", ")")); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java index 9be2aba6451ab1..87b881074f178d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java @@ -59,7 +59,7 @@ public DateTimeV2Literal(DateTimeV2Type dateType, private void roundMicroSecond(int scale) { Preconditions.checkArgument(scale >= 0 && scale <= DateTimeV2Type.MAX_SCALE, - "invalid datetime v2 scale: " + scale); + "invalid datetime v2 scale: %s", scale); double factor = Math.pow(10, 6 - scale); this.microSecond = Math.round(this.microSecond / factor) * (int) factor; @@ -187,11 +187,8 @@ public DateTimeV2Literal roundCeiling(int newScale) { } public DateTimeV2Literal roundFloor(int newScale) { - long newMicroSecond = Double.valueOf( - microSecond / (int) (Math.pow(10, 6 - newScale)) * (Math.pow(10, 6 - newScale))) - .longValue(); - return new DateTimeV2Literal(DateTimeV2Type.of(newScale), year, month, day, hour, minute, - second, newMicroSecond); + // use roundMicroSecond in constructor + return new DateTimeV2Literal(DateTimeV2Type.of(newScale), year, month, day, hour, minute, second, microSecond); } public static Expression fromJavaDateType(LocalDateTime dateTime) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/JsonLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/JsonLiteral.java index 779be0ca54191a..3897239179e167 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/JsonLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/JsonLiteral.java @@ -32,7 +32,6 @@ public class JsonLiteral extends Literal { private static final ObjectMapper MAPPER = new ObjectMapper(); - private final JsonNode jsonNode; private final String value; /** @@ -40,15 +39,17 @@ public class JsonLiteral extends Literal { */ public JsonLiteral(String value) { super(JsonType.INSTANCE); + JsonNode jsonNode; try { jsonNode = MAPPER.readTree(value); } catch (JsonProcessingException e) { throw new AnalysisException("Invalid jsonb literal: '" + value + "'. because " + e.getMessage()); } - if (jsonNode.isMissingNode()) { + if (jsonNode == null || jsonNode.isMissingNode()) { throw new AnalysisException("Invalid jsonb literal: ''"); + } else { + this.value = jsonNode.toString(); } - this.value = jsonNode.toString(); } @Override @@ -57,8 +58,8 @@ public String getStringValue() { } @Override - public JsonNode getValue() { - return jsonNode; + public String getValue() { + return value; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateTableCommand.java index 6f5cf6566733ae..43e3b0ce64164f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateTableCommand.java @@ -107,7 +107,8 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { } else if (i == 0 && dataType.isStringType()) { dataType = VarcharType.createVarcharType(ScalarType.MAX_VARCHAR_LENGTH); } - columnsOfQuery.add(new ColumnDefinition(s.getName(), dataType, s.nullable())); + // if the column is an expression, we set it to nullable, otherwise according to the nullable of the slot. + columnsOfQuery.add(new ColumnDefinition(s.getName(), dataType, !s.isColumnFromTable() || s.nullable())); } createTableInfo.validateCreateTableAsSelect(columnsOfQuery.build(), ctx); CreateTableStmt createTableStmt = createTableInfo.translateToLegacyStmt(); 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 f87b79308c32c3..780ba178f09e97 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 @@ -81,8 +81,8 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync, private final LogicalPlan logicalQuery; private final Optional labelName; + private final boolean isOverwrite; private NereidsPlanner planner; - private boolean isOverwrite; private boolean isTxnBegin = false; /** @@ -129,10 +129,10 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { } String label = this.labelName.orElse(String.format("label_%x_%x", ctx.queryId().hi, ctx.queryId().lo)); - Optional plan = ((Set) planner.getPhysicalPlan() - .collect(node -> node instanceof PhysicalOlapTableSink)).stream().findAny(); + Optional> plan = (planner.getPhysicalPlan() + .>>collect(node -> node instanceof PhysicalOlapTableSink)).stream().findAny(); Preconditions.checkArgument(plan.isPresent(), "insert into command must contain OlapTableSinkNode"); - PhysicalOlapTableSink physicalOlapTableSink = ((PhysicalOlapTableSink) plan.get()); + PhysicalOlapTableSink physicalOlapTableSink = ((PhysicalOlapTableSink) plan.get()); if (isOverwrite) { dealOverwrite(ctx, executor, physicalOlapTableSink); @@ -189,14 +189,15 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { * @param ctx ctx * @param executor executor * @param physicalOlapTableSink physicalOlapTableSink + * * @throws Exception Exception */ - public void dealOverwrite(ConnectContext ctx, StmtExecutor executor, PhysicalOlapTableSink physicalOlapTableSink) - throws Exception { + public void dealOverwrite(ConnectContext ctx, StmtExecutor executor, + PhysicalOlapTableSink physicalOlapTableSink) throws Exception { OlapTable targetTable = physicalOlapTableSink.getTargetTable(); TableName tableName = new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, targetTable.getQualifiedDbName(), targetTable.getName()); - List partitionNames = ((UnboundOlapTableSink) logicalQuery).getPartitions(); + List partitionNames = ((UnboundOlapTableSink) logicalQuery).getPartitions(); if (CollectionUtils.isEmpty(partitionNames)) { partitionNames = Lists.newArrayList(targetTable.getPartitionNames()); } @@ -243,12 +244,11 @@ private void replacePartition(ConnectContext ctx, TableName tableName, List tempPartitionNames, TableName tableName) { try { - UnboundOlapTableSink sink = (UnboundOlapTableSink) logicalQuery; + UnboundOlapTableSink sink = (UnboundOlapTableSink) logicalQuery; UnboundOlapTableSink copySink = new UnboundOlapTableSink<>( sink.getNameParts(), sink.getColNames(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java index 0a0b3fd9a06828..742b2ea801b85b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java @@ -105,7 +105,8 @@ public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuer List selectItems = Lists.newArrayList(); String tableName = tableAlias != null ? tableAlias : targetTable.getName(); for (Column column : targetTable.getFullSchema()) { - if (!column.isVisible()) { + // if it sets sequence column in stream load phase, the sequence map column is null, we query it. + if (!column.isVisible() && !column.isSequenceColumn()) { continue; } if (colNameToExpression.containsKey(column.getName())) { @@ -123,9 +124,12 @@ public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuer logicalQuery = ((LogicalPlan) cte.get().withChildren(logicalQuery)); } + boolean isPartialUpdate = targetTable.getEnableUniqueKeyMergeOnWrite() + && selectItems.size() < targetTable.getColumns().size(); + // make UnboundTableSink return new UnboundOlapTableSink<>(nameParts, ImmutableList.of(), ImmutableList.of(), - ImmutableList.of(), logicalQuery); + ImmutableList.of(), isPartialUpdate, logicalQuery); } private void checkTable(ConnectContext ctx) throws AnalysisException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java index d7d890dda618e7..a3641057a7aae0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java @@ -138,8 +138,8 @@ public int hashCode() { public String toString() { return Utils.toSqlString("LogicalOlapTableSink[" + id.asInt() + "]", "outputExprs", outputExprs, - "database", database, - "targetTable", targetTable, + "database", database.getFullName(), + "targetTable", targetTable.getName(), "cols", cols, "partitionIds", partitionIds, "isPartialUpdate", isPartialUpdate, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java index 41ea6d09ef51c0..d19b9398762412 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.trees.plans.logical; import org.apache.doris.catalog.Type; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.trees.expressions.Alias; @@ -30,7 +31,11 @@ import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.algebra.SetOperation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.StructField; +import org.apache.doris.nereids.types.StructType; import org.apache.doris.nereids.util.TypeCoercionUtils; import com.google.common.base.Preconditions; @@ -137,10 +142,7 @@ private List> castCommonDataTypeOutputs() { for (int i = 0; i < child(0).getOutput().size(); ++i) { Slot left = child(0).getOutput().get(i); Slot right = child(1).getOutput().get(i); - DataType compatibleType = DataType.fromCatalogType(Type.getAssignmentCompatibleType( - left.getDataType().toCatalogDataType(), - right.getDataType().toCatalogDataType(), - false)); + DataType compatibleType = getAssignmentCompatibleType(left.getDataType(), right.getDataType()); Expression newLeft = TypeCoercionUtils.castIfNotSameType(left, compatibleType); Expression newRight = TypeCoercionUtils.castIfNotSameType(right, compatibleType); if (newLeft instanceof Cast) { @@ -211,4 +213,46 @@ public abstract LogicalSetOperation withChildrenAndTheirOutputs( public int getArity() { return children.size(); } + + private DataType getAssignmentCompatibleType(DataType left, DataType right) { + if (left.isNullType()) { + return right; + } + if (right.isNullType()) { + return left; + } + if (left.equals(right)) { + return left; + } + if (left instanceof ArrayType && right instanceof ArrayType) { + return ArrayType.of(getAssignmentCompatibleType( + ((ArrayType) left).getItemType(), ((ArrayType) right).getItemType())); + } + if (left instanceof MapType && right instanceof MapType) { + return MapType.of( + getAssignmentCompatibleType(((MapType) left).getKeyType(), ((MapType) right).getKeyType()), + getAssignmentCompatibleType(((MapType) left).getValueType(), ((MapType) right).getValueType())); + } + if (left instanceof StructType && right instanceof StructType) { + List leftFields = ((StructType) left).getFields(); + List rightFields = ((StructType) right).getFields(); + if (leftFields.size() != rightFields.size()) { + throw new AnalysisException( + "could not get common type for two different struct type " + left + ", " + right); + } + ImmutableList.Builder commonFields = ImmutableList.builder(); + for (int i = 0; i < leftFields.size(); i++) { + boolean nullable = leftFields.get(i).isNullable() || rightFields.get(i).isNullable(); + DataType commonType = getAssignmentCompatibleType( + leftFields.get(i).getDataType(), rightFields.get(i).getDataType()); + StructField commonField = leftFields.get(i).withDataTypeAndNulalble(commonType, nullable); + commonFields.add(commonField); + } + return new StructType(commonFields.build()); + } + return DataType.fromCatalogType(Type.getAssignmentCompatibleType( + left.toCatalogDataType(), + right.toCatalogDataType(), + false)); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFilter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFilter.java index 99a40baba5f81c..4f2f5427d5b299 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFilter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFilter.java @@ -37,6 +37,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; /** * Physical filter plan. @@ -136,9 +137,14 @@ public PhysicalFilter withConjunctsAndChild(Set conjuncts, Pla @Override public String shapeInfo() { StringBuilder builder = new StringBuilder(); - builder.append("filter("); - conjuncts.forEach(conjunct -> builder.append(conjunct.shapeInfo())); - builder.append(")"); + builder.append("filter"); + builder.append( + conjuncts.stream().map(conjunct -> conjunct.shapeInfo()) + .sorted() + .collect(Collectors.joining(" and ", "(", ")"))); + // List strConjuncts = Lists.newArrayList(); + // conjuncts.forEach(conjunct -> strConjuncts.add(conjunct.shapeInfo())); + // builder.append(strConjuncts.stream().sorted().collect(Collectors.joining(" and ", "(", ")"))); return builder.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashJoin.java index bdc14ec3382041..6d9583504eae1c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashJoin.java @@ -48,7 +48,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -274,43 +273,15 @@ public boolean pushDownRuntimeFilter(CascadesContext context, IdGenerator { - @Override - public int compare(Expression e1, Expression e2) { - List ids1 = e1.getInputSlotExprIds() - .stream().sorted(Comparator.comparing(ExprId::asInt)) - .collect(Collectors.toList()); - List ids2 = e2.getInputSlotExprIds() - .stream().sorted(Comparator.comparing(ExprId::asInt)) - .collect(Collectors.toList()); - if (ids1.size() > ids2.size()) { - return 1; - } else if (ids1.size() < ids2.size()) { - return -1; - } else { - for (int i = 0; i < ids1.size(); i++) { - if (ids1.get(i).asInt() > ids2.get(i).asInt()) { - return 1; - } else if (ids1.get(i).asInt() < ids2.get(i).asInt()) { - return -1; - } - } - return 0; - } - } - } - @Override public String shapeInfo() { StringBuilder builder = new StringBuilder(); builder.append("hashJoin[").append(joinType).append("]"); // print sorted hash conjuncts for plan check - hashJoinConjuncts.stream().sorted(new ExprComparator()).forEach(expr -> { - builder.append(expr.shapeInfo()); - }); - otherJoinConjuncts.stream().sorted(new ExprComparator()).forEach(expr -> { - builder.append(expr.shapeInfo()); - }); + builder.append(hashJoinConjuncts.stream().map(conjunct -> conjunct.shapeInfo()) + .sorted().collect(Collectors.joining(" and ", " hashCondition=(", ")"))); + builder.append(otherJoinConjuncts.stream().map(cond -> cond.shapeInfo()) + .sorted().collect(Collectors.joining(" and ", "otherCondition=(", ")"))); return builder.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java index 3c0a7177fc4116..093c87281ef453 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java @@ -157,8 +157,8 @@ public int hashCode() { public String toString() { return Utils.toSqlString("LogicalOlapTableSink[" + id.asInt() + "]", "outputExprs", outputExprs, - "database", database, - "targetTable", targetTable, + "database", database.getFullName(), + "targetTable", targetTable.getName(), "cols", cols, "partitionIds", partitionIds, "singleReplicaLoad", singleReplicaLoad, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructField.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructField.java index 135866738fb192..fed07259be3519 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructField.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructField.java @@ -71,6 +71,10 @@ public StructField withDataType(DataType dataType) { return new StructField(name, dataType, nullable, comment); } + public StructField withDataTypeAndNulalble(DataType dataType, boolean nullable) { + return new StructField(name, dataType, nullable, comment); + } + public org.apache.doris.catalog.StructField toCatalogDataType() { return new org.apache.doris.catalog.StructField( name, dataType.toCatalogDataType(), comment, nullable); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/ColocatePersistInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/ColocatePersistInfo.java index 459be6460524ea..429d4e0e1a6b94 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/ColocatePersistInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/ColocatePersistInfo.java @@ -18,6 +18,7 @@ package org.apache.doris.persist; import org.apache.doris.catalog.ColocateTableIndex.GroupId; +import org.apache.doris.catalog.ReplicaAllocation; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonUtils; @@ -45,29 +46,38 @@ public class ColocatePersistInfo implements Writable { private long tableId; @SerializedName(value = "backendsPerBucketSeq") private Map>> backendsPerBucketSeq = Maps.newHashMap(); + @SerializedName(value = "replicaAlloc") + private ReplicaAllocation replicaAlloc = new ReplicaAllocation(); - private ColocatePersistInfo(GroupId groupId, long tableId, Map>> backendsPerBucketSeq) { + private ColocatePersistInfo(GroupId groupId, long tableId, Map>> backendsPerBucketSeq, + ReplicaAllocation replicaAlloc) { this.groupId = groupId; this.tableId = tableId; this.backendsPerBucketSeq = backendsPerBucketSeq; + this.replicaAlloc = replicaAlloc; } public static ColocatePersistInfo createForAddTable(GroupId groupId, long tableId, Map>> backendsPerBucketSeq) { - return new ColocatePersistInfo(groupId, tableId, backendsPerBucketSeq); + return new ColocatePersistInfo(groupId, tableId, backendsPerBucketSeq, new ReplicaAllocation()); } public static ColocatePersistInfo createForBackendsPerBucketSeq(GroupId groupId, Map>> backendsPerBucketSeq) { - return new ColocatePersistInfo(groupId, -1L, backendsPerBucketSeq); + return new ColocatePersistInfo(groupId, -1L, backendsPerBucketSeq, new ReplicaAllocation()); } public static ColocatePersistInfo createForMarkUnstable(GroupId groupId) { - return new ColocatePersistInfo(groupId, -1L, Maps.newHashMap()); + return new ColocatePersistInfo(groupId, -1L, Maps.newHashMap(), new ReplicaAllocation()); } public static ColocatePersistInfo createForMarkStable(GroupId groupId) { - return new ColocatePersistInfo(groupId, -1L, Maps.newHashMap()); + return new ColocatePersistInfo(groupId, -1L, Maps.newHashMap(), new ReplicaAllocation()); + } + + public static ColocatePersistInfo createForModifyReplicaAlloc(GroupId groupId, ReplicaAllocation replicaAlloc, + Map>> backendsPerBucketSeq) { + return new ColocatePersistInfo(groupId, -1L, backendsPerBucketSeq, replicaAlloc); } public static ColocatePersistInfo read(DataInput in) throws IOException { @@ -87,6 +97,10 @@ public Map>> getBackendsPerBucketSeq() { return backendsPerBucketSeq; } + public ReplicaAllocation getReplicaAlloc() { + return replicaAlloc; + } + @Override public void write(DataOutput out) throws IOException { Text.writeString(out, GsonUtils.GSON.toJson(this)); @@ -129,7 +143,7 @@ public boolean equals(Object obj) { ColocatePersistInfo info = (ColocatePersistInfo) obj; return tableId == info.tableId && groupId.equals(info.groupId) && backendsPerBucketSeq.equals( - info.backendsPerBucketSeq); + info.backendsPerBucketSeq) && replicaAlloc.equals(info.replicaAlloc); } @Override @@ -138,6 +152,7 @@ public String toString() { sb.append("table id: ").append(tableId); sb.append(" group id: ").append(groupId); sb.append(" backendsPerBucketSeq: ").append(backendsPerBucketSeq); + sb.append(" replicaAlloc: ").append(replicaAlloc); return sb.toString(); } } 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 014a3c79feaa17..6e714eb3bfcb9a 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 @@ -605,6 +605,11 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { env.getColocateTableIndex().replayMarkGroupStable(info); break; } + case OperationType.OP_COLOCATE_MOD_REPLICA_ALLOC: { + final ColocatePersistInfo info = (ColocatePersistInfo) journal.getData(); + env.getColocateTableIndex().replayModifyReplicaAlloc(info); + break; + } case OperationType.OP_MODIFY_TABLE_COLOCATE: { final TablePropertyInfo info = (TablePropertyInfo) journal.getData(); env.replayModifyTableColocate(info); @@ -1548,6 +1553,10 @@ public void logTruncateTable(TruncateTableInfo info) { Env.getCurrentEnv().getBinlogManager().addTruncateTable(info, logId); } + public void logColocateModifyRepliaAlloc(ColocatePersistInfo info) { + logEdit(OperationType.OP_COLOCATE_MOD_REPLICA_ALLOC, info); + } + public void logColocateAddTable(ColocatePersistInfo info) { logEdit(OperationType.OP_COLOCATE_ADD_TABLE, info); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index 27cb57d214da9e..c5e784cb9d7eff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -189,6 +189,7 @@ public class OperationType { // modify database/table/tablet/replica meta public static final short OP_SET_REPLICA_VERSION = 141; + public static final short OP_COLOCATE_MOD_REPLICA_ALLOC = 142; // routine load 200 public static final short OP_CREATE_ROUTINE_LOAD_JOB = 200; diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java index 8e6976b5ef2bcf..7e9078df3d9a4a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileQueryScanNode.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.FsBroker; +import org.apache.doris.catalog.FunctionGenTable; import org.apache.doris.catalog.HdfsResource; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.external.ExternalTable; @@ -49,6 +50,7 @@ import org.apache.doris.spi.Split; import org.apache.doris.statistics.StatisticalType; import org.apache.doris.system.Backend; +import org.apache.doris.tablefunction.ExternalFileTableValuedFunction; import org.apache.doris.thrift.TExternalScanRange; import org.apache.doris.thrift.TFileAttributes; import org.apache.doris.thrift.TFileCompressType; @@ -274,11 +276,14 @@ public void createScanRangeLocations() throws UserException { TFileFormatType fileFormatType = getFileFormatType(); params.setFormatType(fileFormatType); boolean isCsvOrJson = Util.isCsvFormat(fileFormatType) || fileFormatType == TFileFormatType.FORMAT_JSON; - if (isCsvOrJson) { + boolean isWal = fileFormatType == TFileFormatType.FORMAT_WAL; + if (isCsvOrJson || isWal) { params.setFileAttributes(getFileAttributes()); if (getLocationType() == TFileType.FILE_STREAM) { params.setFileType(TFileType.FILE_STREAM); - params.setCompressType(TFileCompressType.PLAIN); + FunctionGenTable table = (FunctionGenTable) this.desc.getTable(); + ExternalFileTableValuedFunction tableValuedFunction = (ExternalFileTableValuedFunction) table.getTvf(); + params.setCompressType(tableValuedFunction.getTFileCompressType()); TScanRangeLocations curLocations = newLocations(); TFileRangeDesc rangeDesc = new TFileRangeDesc(); 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 5a427a108010ec..302d3544dedde4 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 @@ -26,6 +26,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.telemetry.Telemetry; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.SessionContext; @@ -87,6 +88,8 @@ public class ConnectContext { protected volatile String traceId; // id for this connection protected volatile int connectionId; + // Timestamp when the connection is make + protected volatile long loginTime; // mysql net protected volatile MysqlChannel mysqlChannel; // state @@ -500,6 +503,10 @@ public void setConnectionId(int connectionId) { this.connectionId = connectionId; } + public void resetLoginTime() { + this.loginTime = System.currentTimeMillis(); + } + public MysqlChannel getMysqlChannel() { return mysqlChannel; } @@ -772,21 +779,30 @@ public Map getResultAttachedInfo() { public class ThreadInfo { public boolean isFull; - public List toRow(long nowMs, boolean showFe) { + public List toRow(int connId, long nowMs, boolean showFe) { List row = Lists.newArrayList(); if (showFe) { row.add(Env.getCurrentEnv().getSelfNode().getHost()); } + if (connId == connectionId) { + row.add("Yes"); + } else { + row.add(""); + } row.add("" + connectionId); row.add(ClusterNamespace.getNameFromFullName(qualifiedUser)); row.add(getMysqlChannel().getRemoteHostPortString()); - row.add(clusterName); + row.add(TimeUtils.longToTimeString(loginTime)); + row.add(defaultCatalog); row.add(ClusterNamespace.getNameFromFullName(currentDb)); row.add(command.toString()); row.add("" + (nowMs - startTime) / 1000); - row.add(""); - if (queryId != null) { - String sql = QeProcessorImpl.INSTANCE.getCurrentQueryByQueryId(queryId); + row.add(state.toString()); + row.add(DebugUtil.printId(queryId)); + if (state.getStateType() == QueryState.MysqlStateType.ERR) { + row.add(state.getErrorMessage()); + } else if (executor != null) { + String sql = executor.getOriginStmtInString(); if (!isFull) { sql = sql.substring(0, Math.min(sql.length(), 100)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java index a89324de390ef6..5090e623a951c1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java @@ -81,6 +81,7 @@ public boolean submit(ConnectContext context) { return false; } context.setConnectionId(nextConnectionId.getAndAdd(1)); + context.resetLoginTime(); return true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java index 9056ac4e802ec5..2af8feda27a5a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java @@ -31,6 +31,7 @@ import org.apache.doris.analysis.AdminSetTableStatusStmt; import org.apache.doris.analysis.AlterCatalogNameStmt; import org.apache.doris.analysis.AlterCatalogPropertyStmt; +import org.apache.doris.analysis.AlterColocateGroupStmt; import org.apache.doris.analysis.AlterColumnStatsStmt; import org.apache.doris.analysis.AlterDatabasePropertyStmt; import org.apache.doris.analysis.AlterDatabaseQuotaStmt; @@ -319,6 +320,8 @@ public static void execute(Env env, DdlStmt ddlStmt) throws Exception { env.getRefreshManager().handleRefreshDb((RefreshDbStmt) ddlStmt); } else if (ddlStmt instanceof AlterResourceStmt) { env.getResourceMgr().alterResource((AlterResourceStmt) ddlStmt); + } else if (ddlStmt instanceof AlterColocateGroupStmt) { + env.getColocateTableIndex().alterColocateGroup((AlterColocateGroupStmt) ddlStmt); } else if (ddlStmt instanceof AlterWorkloadGroupStmt) { env.getWorkloadGroupMgr().alterWorkloadGroup((AlterWorkloadGroupStmt) ddlStmt); } else if (ddlStmt instanceof CreatePolicyStmt) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 52a68b9e047476..8a138aeda44a8a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -689,7 +689,7 @@ public class SessionVariable implements Serializable, Writable { public boolean extractWideRangeExpr = true; @VariableMgr.VarAttr(name = ENABLE_NEREIDS_DML, needForward = true) - public boolean enableNereidsDML = false; + public boolean enableNereidsDML = true; @VariableMgr.VarAttr(name = ENABLE_STRICT_CONSISTENCY_DML, needForward = true) public boolean enableStrictConsistencyDml = false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index bb435c443dd819..47a29ce9620825 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -461,7 +461,7 @@ private void handleShowProcesslist() { .listConnection(ctx.getQualifiedUser(), showStmt.isFull()); long nowMs = System.currentTimeMillis(); for (ConnectContext.ThreadInfo info : threadInfos) { - rowSet.add(info.toRow(nowMs, false)); + rowSet.add(info.toRow(ctx.getConnectionId(), nowMs, false)); } resultSet = new ShowResultSet(showStmt.getMetaData(), rowSet); 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 e0e68180bdec12..26914ba48112b2 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 @@ -2759,6 +2759,13 @@ public ConnectContext getContext() { public OriginStatement getOriginStmt() { return originStmt; } + + public String getOriginStmtInString() { + if (originStmt != null && originStmt.originStmt != null) { + return originStmt.originStmt; + } + return ""; + } } 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 e2a9cc23ab9f02..b06cfc91d14608 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 @@ -1174,8 +1174,9 @@ private TLoadTxnBeginResult loadTxnBeginImpl(TLoadTxnBeginRequest request, Strin if (Strings.isNullOrEmpty(cluster)) { cluster = SystemInfoService.DEFAULT_CLUSTER; } - - if (Strings.isNullOrEmpty(request.getToken())) { + if (request.isSetAuthCode()) { + // TODO(cmy): find a way to check + } else if (Strings.isNullOrEmpty(request.getToken())) { checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(), request.getUserIp(), PrivPredicate.LOAD); } @@ -1901,7 +1902,7 @@ public TStreamLoadPutResult streamLoadPut(TStreamLoadPutRequest request) { if (Config.enable_pipeline_load) { result.setPipelineParams(pipelineStreamLoadPutImpl(request)); } else { - result.setParams(streamLoadPutImpl(request)); + result.setParams(streamLoadPutImpl(request, result)); } } } catch (UserException e) { @@ -2040,7 +2041,9 @@ private void httpStreamPutImpl(TStreamLoadPutRequest request, TStreamLoadPutResu cluster = SystemInfoService.DEFAULT_CLUSTER; } ConnectContext ctx = new ConnectContext(); - if (Strings.isNullOrEmpty(request.getToken())) { + if (request.isSetAuthCode()) { + // TODO(cmy): find a way to check + } else if (Strings.isNullOrEmpty(request.getToken())) { checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(), request.getUserIp(), PrivPredicate.LOAD); } @@ -2057,6 +2060,9 @@ private void httpStreamPutImpl(TStreamLoadPutRequest request, TStreamLoadPutResu NativeInsertStmt parsedStmt = (NativeInsertStmt) SqlParserUtils.getFirstStmt(parser); parsedStmt.setOrigStmt(new OriginStatement(originStmt, 0)); parsedStmt.setUserInfo(ctx.getCurrentUserIdentity()); + if (request.isGroupCommit() && parsedStmt.getLabel() != null) { + throw new AnalysisException("label and group_commit can't be set at the same time"); + } StmtExecutor executor = new StmtExecutor(ctx, parsedStmt); ctx.setExecutor(executor); TQueryOptions tQueryOptions = ctx.getSessionVariable().toThrift(); @@ -2065,7 +2071,6 @@ private void httpStreamPutImpl(TStreamLoadPutRequest request, TStreamLoadPutResu Coordinator coord = new Coordinator(ctx, analyzer, executor.planner()); coord.setLoadMemLimit(request.getExecMemLimit()); coord.setQueryType(TQueryType.LOAD); - QeProcessorImpl.INSTANCE.registerQuery(request.getLoadId(), coord); TExecPlanFragmentParams plan = coord.getStreamLoadPlan(); final long txn_id = parsedStmt.getTransactionId(); @@ -2074,10 +2079,12 @@ private void httpStreamPutImpl(TStreamLoadPutRequest request, TStreamLoadPutResu result.getParams().setTableName(parsedStmt.getTbl()); // The txn_id here is obtained from the NativeInsertStmt result.getParams().setTxnConf(new TTxnParams().setTxnId(txn_id)); - if (parsedStmt.isInnerGroupCommit) { - result.setBaseSchemaVersion(((OlapTable) parsedStmt.getTargetTable()).getBaseSchemaVersion()); + if (parsedStmt.isGroupCommitTvf) { result.getParams().params.setGroupCommit(true); } + result.setDbId(parsedStmt.getTargetTable().getDatabase().getId()); + result.setTableId(parsedStmt.getTargetTable().getId()); + result.setBaseSchemaVersion(((OlapTable) parsedStmt.getTargetTable()).getBaseSchemaVersion()); } catch (UserException e) { LOG.warn("exec sql error", e); throw new UserException("exec sql error" + e); @@ -2087,7 +2094,8 @@ private void httpStreamPutImpl(TStreamLoadPutRequest request, TStreamLoadPutResu } } - private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) throws UserException { + private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request, TStreamLoadPutResult result) + throws UserException { String cluster = request.getCluster(); if (Strings.isNullOrEmpty(cluster)) { cluster = SystemInfoService.DEFAULT_CLUSTER; @@ -2105,6 +2113,9 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) } long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() : 5000; Table table = db.getTableOrMetaException(request.getTbl(), TableType.OLAP); + result.setDbId(db.getId()); + result.setTableId(table.getId()); + result.setBaseSchemaVersion(((OlapTable) table).getBaseSchemaVersion()); return generatePlanFragmentParams(request, db, fullDbName, (OlapTable) table, timeoutMs); } @@ -2130,13 +2141,15 @@ private TExecPlanFragmentParams generatePlanFragmentParams(TStreamLoadPutRequest } StreamLoadPlanner planner = new StreamLoadPlanner(db, table, streamLoadTask); TExecPlanFragmentParams plan = planner.plan(streamLoadTask.getId(), multiTableFragmentInstanceIdIndex); - // add table indexes to transaction state - TransactionState txnState = Env.getCurrentGlobalTransactionMgr() - .getTransactionState(db.getId(), request.getTxnId()); - if (txnState == null) { - throw new UserException("txn does not exist: " + request.getTxnId()); + if (!request.isGroupCommit()) { + // add table indexes to transaction state + TransactionState txnState = Env.getCurrentGlobalTransactionMgr() + .getTransactionState(db.getId(), request.getTxnId()); + if (txnState == null) { + throw new UserException("txn does not exist: " + request.getTxnId()); + } + txnState.addTableIndexes(table); } - txnState.addTableIndexes(table); plan.setTableName(table.getName()); plan.query_options.setFeProcessUuid(ExecuteEnv.getInstance().getProcessUUID()); return plan; diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java index ff55bb1d7a0139..69b002c3cb691e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java @@ -25,10 +25,12 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.HdfsResource; import org.apache.doris.catalog.MapType; +import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; +import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.FeConstants; @@ -94,6 +96,7 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio protected static String DEFAULT_COLUMN_SEPARATOR = ","; protected static final String DEFAULT_LINE_DELIMITER = "\n"; public static final String FORMAT = "format"; + public static final String TABLE_ID = "table_id"; public static final String COLUMN_SEPARATOR = "column_separator"; public static final String LINE_DELIMITER = "line_delimiter"; protected static final String JSON_ROOT = "json_root"; @@ -114,6 +117,7 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio protected static final ImmutableSet FILE_FORMAT_PROPERTIES = new ImmutableSet.Builder() .add(FORMAT) + .add(TABLE_ID) .add(JSON_ROOT) .add(JSON_PATHS) .add(STRIP_OUTER_ARRAY) @@ -157,6 +161,7 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio private boolean fuzzyParse; private boolean trimDoubleQuotes; private int skipLines; + private long tableId; public abstract TFileType getTFileType(); @@ -236,10 +241,14 @@ protected void parseProperties(Map validParams) throws AnalysisE case "avro": this.fileFormatType = TFileFormatType.FORMAT_AVRO; break; + case "wal": + this.fileFormatType = TFileFormatType.FORMAT_WAL; + break; default: throw new AnalysisException("format:" + formatString + " is not supported."); } + tableId = Long.valueOf(validParams.getOrDefault(TABLE_ID, "-1")).longValue(); columnSeparator = validParams.getOrDefault(COLUMN_SEPARATOR, DEFAULT_COLUMN_SEPARATOR); if (Strings.isNullOrEmpty(columnSeparator)) { throw new AnalysisException("column_separator can not be empty."); @@ -401,6 +410,20 @@ public List getTableColumns() throws AnalysisException { throw new AnalysisException("No Alive backends"); } + if (this.fileFormatType == TFileFormatType.FORMAT_WAL) { + List fileColumns = new ArrayList<>(); + Table table = Env.getCurrentInternalCatalog().getTableByTableId(tableId); + List tableColumns = table.getBaseSchema(false); + for (int i = 1; i <= tableColumns.size(); i++) { + fileColumns.add(new Column("c" + i, tableColumns.get(i - 1).getDataType(), true)); + } + Column deleteSignColumn = ((OlapTable) table).getDeleteSignColumn(); + if (deleteSignColumn != null) { + fileColumns.add(new Column("c" + (tableColumns.size() + 1), deleteSignColumn.getDataType(), true)); + } + return fileColumns; + } + TNetworkAddress address = new TNetworkAddress(be.getHost(), be.getBrpcPort()); try { PFetchTableSchemaRequest request = getFetchTableStructureRequest(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java index bb32c826538160..265045d7a6f97f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java @@ -22,10 +22,10 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.thrift.TFileType; +import org.apache.commons.collections.map.CaseInsensitiveMap; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.util.HashMap; import java.util.Map; /** @@ -37,16 +37,16 @@ public class HttpStreamTableValuedFunction extends ExternalFileTableValuedFuncti public static final String NAME = "http_stream"; public HttpStreamTableValuedFunction(Map params) throws AnalysisException { - Map fileParams = new HashMap<>(); + Map fileParams = new CaseInsensitiveMap(); for (String key : params.keySet()) { String lowerKey = key.toLowerCase(); if (!FILE_FORMAT_PROPERTIES.contains(lowerKey)) { throw new AnalysisException(key + " is invalid property"); } - fileParams.put(lowerKey, params.get(key)); + fileParams.put(lowerKey, params.get(key).toLowerCase()); } - String formatString = fileParams.getOrDefault(FORMAT, ""); + String formatString = fileParams.getOrDefault(FORMAT, "").toLowerCase(); if (formatString.equals("parquet") || formatString.equals("avro") || formatString.equals("orc")) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java index fb61fe356d3a65..95f3c19ed27c2b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java @@ -40,6 +40,9 @@ import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; public class PushTask extends AgentTask { private static final Logger LOG = LogManager.getLogger(PushTask.class); @@ -127,33 +130,36 @@ public TPushReq toThrift() { break; case DELETE: List tConditions = new ArrayList(); + Map colNameToColDesc = columnsDesc.stream() + .collect(Collectors.toMap(TColumn::getColumnName, Function.identity())); for (Predicate condition : conditions) { TCondition tCondition = new TCondition(); ArrayList conditionValues = new ArrayList(); + SlotRef slotRef = (SlotRef) condition.getChild(0); + String columnName = slotRef.getColumnName(); + tCondition.setColumnName(columnName); + int uniqueId = colNameToColDesc.get(columnName).getColUniqueId(); + if (uniqueId >= 0) { + tCondition.setColumnUniqueId(uniqueId); + } if (condition instanceof BinaryPredicate) { BinaryPredicate binaryPredicate = (BinaryPredicate) condition; - String columnName = ((SlotRef) binaryPredicate.getChild(0)).getColumnName(); String value = ((LiteralExpr) binaryPredicate.getChild(1)).getStringValue(); Operator op = binaryPredicate.getOp(); - tCondition.setColumnName(columnName); tCondition.setConditionOp(op.toString()); conditionValues.add(value); } else if (condition instanceof IsNullPredicate) { IsNullPredicate isNullPredicate = (IsNullPredicate) condition; - String columnName = ((SlotRef) isNullPredicate.getChild(0)).getColumnName(); String op = "IS"; String value = "NULL"; if (isNullPredicate.isNotNull()) { value = "NOT NULL"; } - tCondition.setColumnName(columnName); tCondition.setConditionOp(op); conditionValues.add(value); } else if (condition instanceof InPredicate) { InPredicate inPredicate = (InPredicate) condition; - String columnName = ((SlotRef) inPredicate.getChild(0)).getColumnName(); String op = inPredicate.isNotIn() ? "!*=" : "*="; - tCondition.setColumnName(columnName); tCondition.setConditionOp(op); for (int i = 1; i <= inPredicate.getInElementNum(); i++) { conditionValues.add(inPredicate.getChild(i).getStringValue()); diff --git a/fe/fe-core/src/main/jflex/sql_scanner.flex b/fe/fe-core/src/main/jflex/sql_scanner.flex index 0f8eaa5d9bc4d4..f743d5edd4e6b0 100644 --- a/fe/fe-core/src/main/jflex/sql_scanner.flex +++ b/fe/fe-core/src/main/jflex/sql_scanner.flex @@ -147,6 +147,7 @@ import org.apache.doris.qe.SqlModeHelper; keywordMap.put("clusters", new Integer(SqlParserSymbols.KW_CLUSTERS)); keywordMap.put("collate", new Integer(SqlParserSymbols.KW_COLLATE)); keywordMap.put("collation", new Integer(SqlParserSymbols.KW_COLLATION)); + keywordMap.put("colocate", new Integer(SqlParserSymbols.KW_COLOCATE)); keywordMap.put("column", new Integer(SqlParserSymbols.KW_COLUMN)); keywordMap.put("columns", new Integer(SqlParserSymbols.KW_COLUMNS)); keywordMap.put("comment", new Integer(SqlParserSymbols.KW_COMMENT)); diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java index a3ddd9991a89e3..d681376d24061b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java @@ -17,6 +17,7 @@ package org.apache.doris.alter; +import org.apache.doris.analysis.AlterColocateGroupStmt; import org.apache.doris.analysis.AlterTableStmt; import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateMaterializedViewStmt; @@ -26,6 +27,8 @@ import org.apache.doris.analysis.DateLiteral; import org.apache.doris.analysis.DropResourceStmt; import org.apache.doris.analysis.ShowCreateMaterializedViewStmt; +import org.apache.doris.catalog.ColocateGroupSchema; +import org.apache.doris.catalog.ColocateTableIndex.GroupId; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.DataProperty; import org.apache.doris.catalog.Database; @@ -36,10 +39,13 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.ReplicaAllocation; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.catalog.Type; +import org.apache.doris.clone.RebalancerTestUtil; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; @@ -47,6 +53,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.DdlExecutor; import org.apache.doris.qe.ShowExecutor; import org.apache.doris.resource.Tag; import org.apache.doris.system.Backend; @@ -70,18 +77,36 @@ public class AlterTest { private static String runningDir = "fe/mocked/AlterTest/" + UUID.randomUUID().toString() + "/"; private static ConnectContext connectContext; - private static Backend be; + + private static Map backendTags; @BeforeClass public static void beforeClass() throws Exception { FeConstants.runningUnitTest = true; FeConstants.default_scheduler_interval_millisecond = 100; + FeConstants.tablet_checker_interval_ms = 100; + FeConstants.tablet_checker_interval_ms = 100; Config.dynamic_partition_check_interval_seconds = 1; Config.disable_storage_medium_check = true; Config.enable_storage_policy = true; - UtFrameUtils.createDorisCluster(runningDir); + Config.disable_balance = true; + Config.schedule_batch_size = 400; + Config.schedule_slot_num_per_hdd_path = 100; + UtFrameUtils.createDorisClusterWithMultiTag(runningDir, 5); + + List backends = Env.getCurrentSystemInfo().getIdToBackend().values().asList(); + + Map tagMap = Maps.newHashMap(); + tagMap.put(Tag.TYPE_LOCATION, "group_a"); + backends.get(2).setTagMap(tagMap); + backends.get(3).setTagMap(tagMap); + + tagMap = Maps.newHashMap(); + tagMap.put(Tag.TYPE_LOCATION, "group_b"); + backends.get(4).setTagMap(tagMap); - be = Env.getCurrentSystemInfo().getIdToBackend().values().asList().get(0); + backendTags = Maps.newHashMap(); + backends.forEach(be -> backendTags.put(be.getId(), be.getLocationTag())); // create connect context connectContext = UtFrameUtils.createDefaultCtx(); @@ -435,21 +460,16 @@ public void testConflictAlterOperations() throws Exception { // set un-partitioned table's real replication num // first we need to change be's tag - Map originTagMap = be.getTagMap(); - Map tagMap = Maps.newHashMap(); - tagMap.put(Tag.TYPE_LOCATION, "group1"); - be.setTagMap(tagMap); OlapTable tbl2 = (OlapTable) db.getTableOrMetaException("tbl2"); Partition partition = tbl2.getPartition(tbl2.getName()); Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl2.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum())); - stmt = "alter table test.tbl2 set ('replication_allocation' = 'tag.location.group1:1');"; + stmt = "alter table test.tbl2 set ('replication_allocation' = 'tag.location.group_a:1');"; alterTable(stmt, false); Assert.assertEquals((short) 1, (short) tbl2.getPartitionInfo().getReplicaAllocation(partition.getId()) - .getReplicaNumByTag(Tag.createNotCheck(Tag.TYPE_LOCATION, "group1"))); + .getReplicaNumByTag(Tag.createNotCheck(Tag.TYPE_LOCATION, "group_a"))); Assert.assertEquals((short) 1, (short) tbl2.getTableProperty().getReplicaAllocation() - .getReplicaNumByTag(Tag.createNotCheck(Tag.TYPE_LOCATION, "group1"))); - be.setTagMap(originTagMap); + .getReplicaNumByTag(Tag.createNotCheck(Tag.TYPE_LOCATION, "group_a"))); Thread.sleep(5000); // sleep to wait dynamic partition scheduler run // add partition without set replication num, and default num is 3. @@ -1202,6 +1222,145 @@ public void testDropInUseResource() throws Exception { Env.getCurrentEnv().getResourceMgr().dropResource(stmt); } + @Test + public void testModifyColocateGroupReplicaAlloc() throws Exception { + Config.enable_round_robin_create_tablet = true; + + createTable("CREATE TABLE test.col_tbl0\n" + "(\n" + " k1 date,\n" + " k2 int,\n" + " v1 int \n" + + ") ENGINE=OLAP\n" + "UNIQUE KEY (k1,k2)\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 4\n" + + "PROPERTIES('replication_num' = '2', 'colocate_with' = 'mod_group_0');"); + + createTable("CREATE TABLE test.col_tbl1\n" + "(\n" + " k1 date,\n" + " k2 int,\n" + " v1 int \n" + + ") ENGINE=OLAP\n" + "UNIQUE KEY (k1,k2)\n" + "PARTITION BY RANGE(k1)\n" + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + ")\n" + "DISTRIBUTED BY HASH(k2) BUCKETS 4\n" + + "PROPERTIES('replication_num' = '2', 'colocate_with' = 'mod_group_1');"); + + createTable("CREATE TABLE test.col_tbl2 (\n" + + "`uuid` varchar(255) NULL,\n" + + "`action_datetime` date NULL\n" + + ")\n" + + "DUPLICATE KEY(uuid)\n" + + "PARTITION BY RANGE(action_datetime)()\n" + + "DISTRIBUTED BY HASH(uuid) BUCKETS 4\n" + + "PROPERTIES\n" + + "(\n" + + "\"replication_num\" = \"2\",\n" + + "\"colocate_with\" = \"mod_group_2\",\n" + + "\"dynamic_partition.enable\" = \"true\",\n" + + "\"dynamic_partition.time_unit\" = \"DAY\",\n" + + "\"dynamic_partition.end\" = \"2\",\n" + + "\"dynamic_partition.prefix\" = \"p\",\n" + + "\"dynamic_partition.buckets\" = \"4\",\n" + + "\"dynamic_partition.replication_num\" = \"2\"\n" + + ");\n"); + + Env env = Env.getCurrentEnv(); + Database db = env.getInternalCatalog().getDbOrMetaException("default_cluster:test"); + OlapTable tbl2 = (OlapTable) db.getTableOrMetaException("col_tbl2"); + for (int j = 0; true; j++) { + Thread.sleep(2000); + if (tbl2.getAllPartitions().size() > 0) { + break; + } + if (j >= 5) { + Assert.assertTrue("dynamic table not create partition", false); + } + } + + RebalancerTestUtil.updateReplicaPathHash(); + + ReplicaAllocation newReplicaAlloc = new ReplicaAllocation(); + newReplicaAlloc.put(Tag.DEFAULT_BACKEND_TAG, (short) 1); + newReplicaAlloc.put(Tag.create(Tag.TYPE_LOCATION, "group_a"), (short) 1); + newReplicaAlloc.put(Tag.create(Tag.TYPE_LOCATION, "group_b"), (short) 1); + + for (int i = 0; i < 3; i++) { + String groupName = "mod_group_" + i; + String sql = "alter colocate group test." + groupName + + " set ( 'replication_allocation' = '" + newReplicaAlloc.toCreateStmt() + "')"; + String fullGroupName = GroupId.getFullGroupName(db.getId(), groupName); + AlterColocateGroupStmt stmt = (AlterColocateGroupStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext); + DdlExecutor.execute(env, stmt); + + ColocateGroupSchema groupSchema = env.getColocateTableIndex().getGroupSchema(fullGroupName); + Assert.assertNotNull(groupSchema); + Assert.assertEquals(newReplicaAlloc, groupSchema.getReplicaAlloc()); + + OlapTable tbl = (OlapTable) db.getTableOrMetaException("col_tbl" + i); + Assert.assertEquals(newReplicaAlloc, tbl.getDefaultReplicaAllocation()); + for (Partition partition : tbl.getAllPartitions()) { + Assert.assertEquals(newReplicaAlloc, + tbl.getPartitionInfo().getReplicaAllocation(partition.getId())); + } + + if (i == 2) { + Assert.assertEquals(newReplicaAlloc, + tbl.getTableProperty().getDynamicPartitionProperty().getReplicaAllocation()); + } + } + + Config.enable_round_robin_create_tablet = false; + + for (int k = 0; true; k++) { + Thread.sleep(1000); // sleep to wait dynamic partition scheduler run + boolean allStable = true; + for (int i = 0; i < 3; i++) { + String fullGroupName = GroupId.getFullGroupName(db.getId(), "mod_group_" + i); + ColocateGroupSchema groupSchema = env.getColocateTableIndex().getGroupSchema(fullGroupName); + Assert.assertNotNull(groupSchema); + + if (env.getColocateTableIndex().isGroupUnstable(groupSchema.getGroupId())) { + allStable = false; + if (k >= 120) { + Assert.assertTrue(fullGroupName + " is unstable", false); + } + continue; + } + + Map backendReplicaNum = Maps.newHashMap(); + OlapTable tbl = (OlapTable) db.getTableOrMetaException("col_tbl" + i); + int tabletNum = 0; + for (Partition partition : tbl.getAllPartitions()) { + for (MaterializedIndex idx : partition.getMaterializedIndices( + MaterializedIndex.IndexExtState.VISIBLE)) { + for (Tablet tablet : idx.getTablets()) { + Map allocMap = Maps.newHashMap(); + tabletNum++; + for (Replica replica : tablet.getReplicas()) { + long backendId = replica.getBackendId(); + Tag tag = backendTags.get(backendId); + Assert.assertNotNull(tag); + short oldNum = allocMap.getOrDefault(tag, (short) 0); + allocMap.put(tag, (short) (oldNum + 1)); + backendReplicaNum.put(backendId, backendReplicaNum.getOrDefault(backendId, 0) + 1); + } + Assert.assertEquals(newReplicaAlloc.getAllocMap(), allocMap); + } + } + } + + Assert.assertTrue(tabletNum > 0); + + for (Map.Entry entry : backendReplicaNum.entrySet()) { + long backendId = entry.getKey(); + int replicaNum = entry.getValue(); + Tag tag = backendTags.get(backendId); + int sameTagReplicaNum = tabletNum * newReplicaAlloc.getAllocMap().getOrDefault(tag, (short) 0); + int sameTagBeNum = (int) (backendTags.values().stream().filter(t -> t.equals(tag)).count()); + Assert.assertEquals("backend " + backendId + " failed: " + " all backend replica num: " + + backendReplicaNum + ", all backend tag: " + backendTags, + sameTagReplicaNum / sameTagBeNum, replicaNum); + } + } + + if (allStable) { + break; + } + } + } + @Test public void testShowMV() throws Exception { createMV("CREATE MATERIALIZED VIEW test_mv as select k1 from test.show_test group by k1;", false); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicateSqlTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicateSqlTest.java new file mode 100644 index 00000000000000..86f06cacbed4ca --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicateSqlTest.java @@ -0,0 +1,99 @@ +// 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.rules.expression.rules; + +import org.apache.doris.nereids.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.utframe.TestWithFeService; + +import org.junit.jupiter.api.Test; + +class SimplifyComparisonPredicateSqlTest extends TestWithFeService implements MemoPatternMatchSupported { + + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + connectContext.setDatabase("default_cluster:test"); + createTables( + "CREATE TABLE IF NOT EXISTS `log_items_test` (\n" + + " a DATETIME(0) NOT NULL,\n" + + " b decimal(10,2)\n" + + " ) ENGINE=OLAP\n" + + " UNIQUE KEY (`a`)\n" + + " DISTRIBUTED BY HASH(`a`) BUCKETS 120\n" + + " PROPERTIES (\n" + + " \"replication_num\" = \"1\",\n" + + " \"in_memory\" = \"false\",\n" + + " \"compression\" = \"LZ4\",\n" + + " \"storage_cooldown_time\" = \"9999-12-31 23:59:59\",\n" + + " \"enable_unique_key_merge_on_write\" = \"true\"\n" + + " );" + ); + } + + @Test + void testSql() { + PlanChecker.from(connectContext) + .analyze("select * from log_items_test where a < '2023-06-15 23:59:59.999' and b < 111.111;") + .rewrite() + .matches( + logicalFilter() + .when(f -> f.getConjuncts().stream().anyMatch(e -> e.toSql().equals("(a < 2023-06-16 00:00:00)"))) + .when(f -> f.getConjuncts().stream().anyMatch(e -> e.toSql().equals("(b < 111.12)"))) + ) + .printlnTree(); + + PlanChecker.from(connectContext) + .analyze("select * from log_items_test where a <= '2023-06-15 23:59:59.999' and b <= 111.111;") + .rewrite() + .matches( + logicalFilter() + .when(f -> f.getConjuncts().stream().anyMatch(e -> e.toSql().equals("(a <= 2023-06-16 00:00:00)"))) + .when(f -> f.getConjuncts().stream().anyMatch(e -> e.toSql().equals("(b <= 111.11)"))) + ) + .printlnTree(); + + PlanChecker.from(connectContext) + .analyze("select * from log_items_test where a = '2023-06-15 23:59:59.999' and b = 111.111;") + .rewrite() + .matches( + logicalEmptyRelation() + ) + .printlnTree(); + + PlanChecker.from(connectContext) + .analyze("select * from log_items_test where a > '2023-06-15 23:59:59.999' and b > 111.111;") + .rewrite() + .matches( + logicalFilter() + .when(f -> f.getConjuncts().stream().anyMatch(e -> e.toSql().equals("(a > 2023-06-16 00:00:00)"))) + .when(f -> f.getConjuncts().stream().anyMatch(e -> e.toSql().equals("(b > 111.11)"))) + ) + .printlnTree(); + + PlanChecker.from(connectContext) + .analyze("select * from log_items_test where a >= '2023-06-15 23:59:59.999' and b >= 111.111;") + .rewrite() + .matches( + logicalFilter() + .when(f -> f.getConjuncts().stream().anyMatch(e -> e.toSql().equals("(a >= 2023-06-16 00:00:00)"))) + .when(f -> f.getConjuncts().stream().anyMatch(e -> e.toSql().equals("(b >= 111.12)"))) + ) + .printlnTree(); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicateTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicateTest.java index b06d914b65b635..6735e31192fd4a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicateTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicateTest.java @@ -36,10 +36,10 @@ import org.junit.jupiter.api.Test; class SimplifyComparisonPredicateTest extends ExpressionRewriteTestHelper { - @Test void testSimplifyComparisonPredicateRule() { - executor = new ExpressionRuleExecutor(ImmutableList.of(SimplifyCastRule.INSTANCE, SimplifyComparisonPredicate.INSTANCE)); + executor = new ExpressionRuleExecutor( + ImmutableList.of(SimplifyCastRule.INSTANCE, SimplifyComparisonPredicate.INSTANCE)); Expression dtv2 = new DateTimeV2Literal(1, 1, 1, 1, 1, 1, 0); Expression dt = new DateTimeLiteral(1, 1, 1, 1, 1, 1); @@ -67,10 +67,10 @@ void testSimplifyComparisonPredicateRule() { // DateTimeV2 -> Date assertRewrite( new GreaterThan(new Cast(d, DateTimeV2Type.SYSTEM_DEFAULT), dtv2), - new GreaterThan(new Cast(d, DateTimeType.INSTANCE), dt)); + new GreaterThan(new Cast(d, DateTimeType.INSTANCE), new DateTimeLiteral(1, 1, 1, 0, 0, 0))); assertRewrite( new LessThan(new Cast(d, DateTimeV2Type.SYSTEM_DEFAULT), dtv2), - new LessThan(new Cast(d, DateTimeType.INSTANCE), dt)); + new LessThan(new Cast(d, DateTimeType.INSTANCE), new DateTimeLiteral(1, 1, 2, 0, 0, 0))); assertRewrite( new EqualTo(new Cast(d, DateTimeV2Type.SYSTEM_DEFAULT), dtv2), new EqualTo(new Cast(d, DateTimeV2Type.SYSTEM_DEFAULT), dtv2)); @@ -111,4 +111,19 @@ void testDateTimeV2CmpDateTimeV2() { rewrittenExpression = executor.rewrite(typeCoercion(expression), context); Assertions.assertEquals(left.getDataType(), rewrittenExpression.child(0).getDataType()); } + + @Test + void testRound() { + executor = new ExpressionRuleExecutor( + ImmutableList.of(SimplifyCastRule.INSTANCE, SimplifyComparisonPredicate.INSTANCE)); + + Expression left = new Cast(new DateTimeLiteral("2021-01-02 00:00:00.00"), DateTimeV2Type.of(1)); + Expression right = new DateTimeV2Literal("2021-01-01 23:59:59.99"); + // (cast(2021-01-02 00:00:00.00 as DATETIMEV2(1)) > 2021-01-01 23:59:59.99) + Expression expression = new GreaterThan(left, right); + Expression rewrittenExpression = executor.rewrite(typeCoercion(expression), context); + + // right should round to be 2021-01-02 00:00:00.00 + Assertions.assertEquals(new DateTimeV2Literal("2021-01-02 00:00:00"), rewrittenExpression.child(1)); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/TableFunctionPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/TableFunctionPlanTest.java index 014b68477d2c31..52fa95486eabc6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/TableFunctionPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/TableFunctionPlanTest.java @@ -78,7 +78,7 @@ public void normalTableFunction() throws Exception { Assert.assertTrue( explainString.contains("table function: explode_split(`default_cluster:db1`.`tbl1`.`k2`, ',')")); Assert.assertTrue(explainString.contains("tuple ids: 0 1")); - Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=16}")); + Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=32}")); Assert.assertTrue(explainString.contains("SlotDescriptor{id=1, col=e1, colUniqueId=-1, type=varchar")); } @@ -94,7 +94,7 @@ public void withoutOutputExplodeColumn() throws Exception { Assert.assertTrue( explainString.contains("table function: explode_split(`default_cluster:db1`.`tbl1`.`k2`, ',')")); Assert.assertTrue(explainString.contains("tuple ids: 0 1")); - Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=16}")); + Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=32}")); Assert.assertTrue(explainString.contains("SlotDescriptor{id=1, col=e1, colUniqueId=-1, type=varchar")); } @@ -115,7 +115,7 @@ public void groupByExplodeColumn() throws Exception { Assert.assertTrue( explainString.contains("table function: explode_split(`default_cluster:db1`.`tbl1`.`k2`, ',')")); Assert.assertTrue(explainString.contains("tuple ids: 0 1")); - Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=16}")); + Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=32}")); Assert.assertTrue(explainString.contains("SlotDescriptor{id=1, col=e1, colUniqueId=-1, type=varchar")); // group by tuple Assert.assertTrue(explainString.contains("TupleDescriptor{id=2, tbl=null, byteSize=32}")); @@ -134,7 +134,7 @@ public void whereExplodeColumn() throws Exception { explainString.contains("table function: explode_split(`default_cluster:db1`.`tbl1`.`k2`, ',')")); Assert.assertTrue(explainString.contains("PREDICATES: `e1` = '1'")); Assert.assertTrue(explainString.contains("tuple ids: 0 1")); - Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=16}")); + Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=32}")); Assert.assertTrue(explainString.contains("SlotDescriptor{id=1, col=e1, colUniqueId=-1, type=varchar")); } @@ -150,7 +150,7 @@ public void whereNormalColumn() throws Exception { Assert.assertTrue( explainString.contains("table function: explode_split(`default_cluster:db1`.`tbl1`.`k2`, ',')")); Assert.assertTrue(explainString.contains("tuple ids: 0 1")); - Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=16}")); + Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp, byteSize=32}")); Assert.assertTrue(explainString.contains("SlotDescriptor{id=1, col=e1, colUniqueId=-1, type=varchar")); Assert.assertTrue(UtFrameUtils.checkPlanResultContainsNode(explainString, 0, "OlapScanNode")); Assert.assertTrue(explainString.contains("PREDICATES: `k1` = 1")); @@ -170,10 +170,10 @@ public void testMultiLateralView() throws Exception { "table function: explode_split(`default_cluster:db1`.`tbl1`.`k2`, ',') explode_split(`default_cluster:db1`.`tbl1`.`k2`, ',')")); Assert.assertTrue(explainString.contains("lateral view tuple id: 1 2")); // lateral view 2 tuple - Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp2, byteSize=16}")); + Assert.assertTrue(explainString.contains("TupleDescriptor{id=1, tbl=tmp2, byteSize=32}")); Assert.assertTrue(explainString.contains("SlotDescriptor{id=1, col=e2, colUniqueId=-1, type=varchar")); // lateral view 1 tuple - Assert.assertTrue(explainString.contains("TupleDescriptor{id=2, tbl=tmp1, byteSize=16}")); + Assert.assertTrue(explainString.contains("TupleDescriptor{id=2, tbl=tmp1, byteSize=32}")); Assert.assertTrue(explainString.contains("SlotDescriptor{id=2, col=e1, colUniqueId=-1, type=varchar")); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/ConnectContextTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/ConnectContextTest.java index ee9f605693785d..a15bdd8178ac5d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/ConnectContextTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/ConnectContextTest.java @@ -99,19 +99,25 @@ public void testNormal() { ctx.setCommand(MysqlCommand.COM_PING); Assert.assertEquals(MysqlCommand.COM_PING, ctx.getCommand()); + // LoginTime + ctx.loginTime = 1694002396223L; + // Thread info Assert.assertNotNull(ctx.toThreadInfo(false)); - List row = ctx.toThreadInfo(false).toRow(1000, false); - Assert.assertEquals(9, row.size()); - Assert.assertEquals("101", row.get(0)); - Assert.assertEquals("testUser", row.get(1)); - Assert.assertEquals("", row.get(2)); - Assert.assertEquals("testCluster", row.get(3)); - Assert.assertEquals("testDb", row.get(4)); - Assert.assertEquals("Ping", row.get(5)); - Assert.assertEquals("1", row.get(6)); - Assert.assertEquals("", row.get(7)); - Assert.assertEquals("", row.get(8)); + List row = ctx.toThreadInfo(false).toRow(101, 1000, false); + Assert.assertEquals(12, row.size()); + Assert.assertEquals("Yes", row.get(0)); + Assert.assertEquals("101", row.get(1)); + Assert.assertEquals("testUser", row.get(2)); + Assert.assertEquals("", row.get(3)); + Assert.assertEquals("2023-09-06 20:13:16", row.get(4)); + Assert.assertEquals("internal", row.get(5)); + Assert.assertEquals("testDb", row.get(6)); + Assert.assertEquals("Ping", row.get(7)); + Assert.assertEquals("1", row.get(8)); + Assert.assertEquals("OK", row.get(9)); + Assert.assertEquals("", row.get(10)); + Assert.assertEquals("", row.get(11)); // Start time Assert.assertEquals(0, ctx.getStartTime()); 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 407171a69c7ec9..8bb8581fd85220 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 @@ -255,12 +255,16 @@ public static void createDorisClusterWithMultiTag(String runningDir, int backend FeConstants.runningUnitTest = true; FeConstants.enableInternalSchemaDb = false; int feRpcPort = startFEServer(runningDir); + List bes = Lists.newArrayList(); for (int i = 0; i < backendNum; i++) { String host = "127.0.0." + (i + 1); createBackend(host, feRpcPort); } + System.out.println("after create backend"); + checkBEHeartbeat(bes); // sleep to wait first heartbeat - Thread.sleep(6000); + // Thread.sleep(6000); + System.out.println("after create backend2"); } public static Backend createBackend(String beHost, int feRpcPort) throws IOException, InterruptedException { @@ -296,6 +300,7 @@ beHttpPort, beArrowFlightSqlPort, new DefaultHeartbeatServiceImpl(beThriftPort, diskInfo1.setTotalCapacityB(1000000); diskInfo1.setAvailableCapacityB(500000); diskInfo1.setDataUsedCapacityB(480000); + diskInfo1.setPathHash(be.getId()); disks.put(diskInfo1.getRootPath(), diskInfo1); be.setDisks(ImmutableMap.copyOf(disks)); be.setAlive(true); diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 6ef2d512ac43a5..901acca70d50f0 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -639,6 +639,7 @@ struct TStreamLoadPutRequest { // only valid when file type is CSV 52: optional i8 escape 53: optional bool memtable_on_sink_node; + 54: optional bool group_commit } struct TStreamLoadPutResult { @@ -648,6 +649,8 @@ struct TStreamLoadPutResult { 3: optional PaloInternalService.TPipelineFragmentParams pipeline_params // used for group commit 4: optional i64 base_schema_version + 5: optional i64 db_id + 6: optional i64 table_id } struct TStreamLoadMultiTablePutResult { @@ -666,6 +669,16 @@ struct TStreamLoadWithLoadStatusResult { 6: optional i64 unselected_rows } +struct TCheckWalRequest { + 1: optional i64 wal_id + 2: optional i64 db_id +} + +struct TCheckWalResult { + 1: optional Status.TStatus status + 2: optional bool need_recovery +} + struct TKafkaRLTaskProgress { 1: required map partitionCmtOffset } diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 2588996cd49102..cc7102e408ceb9 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -445,6 +445,8 @@ struct TExecPlanFragmentParams { // scan node id -> scan range params, only for external file scan 24: optional map file_scan_params + + 25: optional i64 wal_id } struct TExecPlanFragmentParamsList { diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index 11c72e5ae5f628..3cc1c56911fe2c 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -119,6 +119,7 @@ enum TFileFormatType { FORMAT_AVRO, FORMAT_CSV_LZ4BLOCK, FORMAT_CSV_SNAPPYBLOCK, + FORMAT_WAL, } // In previous versions, the data compression format and file format were stored together, as TFileFormatType, diff --git a/regression-test/data/correctness/test_time_diff_microseconds.out b/regression-test/data/correctness/test_time_diff_microseconds.out index dbeeb067f2643b..cf0f6e93abe947 100644 --- a/regression-test/data/correctness/test_time_diff_microseconds.out +++ b/regression-test/data/correctness/test_time_diff_microseconds.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select1 -- 67:19:00.0000 -24:00:00.5140 +24:00:00.5150 -437:00:00.7683 -- !select2 -- @@ -15,7 +15,7 @@ -- !select5 -- 67:19:00.0000 -24:00:00.5140 +24:00:00.5150 -437:00:00.7683 -- !select6 -- diff --git a/regression-test/data/correctness/test_time_function.out b/regression-test/data/correctness/test_time_function.out index dcdc6da37a3af1..635488787909dc 100644 --- a/regression-test/data/correctness/test_time_function.out +++ b/regression-test/data/correctness/test_time_function.out @@ -1,13 +1,90 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !select1 -- +-- !select -- 16:32:18 --- !select2 -- +-- !select -- 16:32:18 --- !select3 -- +-- !select -- +12:34:56 + +-- !select -- +00:34:56 + +-- !select -- +00:00:56 + +-- !select -- +12:34:56 + +-- !select -- +34:56:00 + +-- !select -- +\N + +-- !select -- +12:34:56 + +-- !select -- +00:34:56 + +-- !select -- +00:00:56 + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- 16:32:18 --- !select4 -- +-- !select -- 16:32:18 +-- !select -- +12:34:56 + +-- !select -- +00:34:56 + +-- !select -- +00:00:56 + +-- !select -- +12:34:56 + +-- !select -- +34:56:00 + +-- !select -- +\N + +-- !select -- +12:34:56 + +-- !select -- +00:34:56 + +-- !select -- +00:00:56 + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N \ No newline at end of file diff --git a/regression-test/data/correctness_p0/test_inlineview_with_lateralview.out b/regression-test/data/correctness_p0/test_inlineview_with_lateralview.out new file mode 100644 index 00000000000000..4fb3a27603feb1 --- /dev/null +++ b/regression-test/data/correctness_p0/test_inlineview_with_lateralview.out @@ -0,0 +1,8 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +5 +5 +5 +5 +5 + diff --git a/regression-test/data/inverted_index_p0/test_tokenize.out b/regression-test/data/inverted_index_p0/test_tokenize.out index 731ae4249eb8f6..c4fa3915b491d4 100644 --- a/regression-test/data/inverted_index_p0/test_tokenize.out +++ b/regression-test/data/inverted_index_p0/test_tokenize.out @@ -19,3 +19,6 @@ ["人民", "得到", "更多", "实惠"] ["陕西", "陕西省", "西安", "西安市", "高新", "高新区", "新区", "创业", "业大", "大厦", "A", "座", "手机", "手机号", "手机号码", "机号码", "号码", "12345678901", "邮箱", "12345678", "qq", "com", "ip", "information", "created", "automatically"] +-- !tokenize_sql -- +["get", "images", "hm", "bg", "jpg", "http", "1", "0", "test", "abc", "bcd"] + diff --git a/regression-test/data/load_p0/http_stream/test_compress.csv.bz2 b/regression-test/data/load_p0/http_stream/test_compress.csv.bz2 new file mode 100644 index 00000000000000..b2fd1fcfbe6af1 Binary files /dev/null and b/regression-test/data/load_p0/http_stream/test_compress.csv.bz2 differ diff --git a/regression-test/data/load_p0/http_stream/test_compress.csv.gz b/regression-test/data/load_p0/http_stream/test_compress.csv.gz new file mode 100644 index 00000000000000..a330a2b07194dd Binary files /dev/null and b/regression-test/data/load_p0/http_stream/test_compress.csv.gz differ diff --git a/regression-test/data/load_p0/http_stream/test_compress.csv.lz4 b/regression-test/data/load_p0/http_stream/test_compress.csv.lz4 new file mode 100644 index 00000000000000..76955306d8ae66 Binary files /dev/null and b/regression-test/data/load_p0/http_stream/test_compress.csv.lz4 differ diff --git a/regression-test/data/load_p0/http_stream/test_group_commit_http_stream.out b/regression-test/data/load_p0/http_stream/test_group_commit_http_stream.out new file mode 100644 index 00000000000000..b45fc6f7140dc3 --- /dev/null +++ b/regression-test/data/load_p0/http_stream/test_group_commit_http_stream.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +5 e -1 +5 e 50 +6 f -1 +6 f -1 +6 f 60 +7 e 70 +8 f 80 + diff --git a/regression-test/data/load_p0/http_stream/test_http_stream_compress.out b/regression-test/data/load_p0/http_stream/test_http_stream_compress.out new file mode 100644 index 00000000000000..80a422c7e72de7 --- /dev/null +++ b/regression-test/data/load_p0/http_stream/test_http_stream_compress.out @@ -0,0 +1,2506 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +0 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +3 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +4 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +5 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +6 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +7 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +8 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +9 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +10 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +11 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +12 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +13 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +14 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +15 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +16 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +17 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +18 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +19 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +20 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +21 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +22 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +23 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +24 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +25 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +26 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +27 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +28 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +29 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +30 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +31 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +32 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +33 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +34 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +35 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +36 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +37 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +38 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +39 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +40 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +41 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +42 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +43 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +44 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +45 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +46 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +47 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +48 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +49 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +50 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +51 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +52 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +53 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +54 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +55 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +56 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +57 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +58 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +59 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +60 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +61 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +62 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +63 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +64 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +65 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +66 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +67 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +68 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +69 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +70 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +71 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +72 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +73 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +74 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +75 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +76 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +77 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +78 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +79 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +80 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +81 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +82 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +83 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +84 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +85 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +86 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +87 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +88 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +89 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +90 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +91 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +92 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +93 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +94 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +95 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +96 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +97 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +98 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +99 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +100 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +101 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +102 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +103 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +104 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +105 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +106 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +107 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +108 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +109 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +110 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +111 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +112 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +113 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +114 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +115 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +116 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +117 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +118 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +119 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +120 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +121 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +122 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +123 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +124 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +125 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +126 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +127 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +128 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +129 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +130 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +131 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +132 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +133 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +134 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +135 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +136 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +137 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +138 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +139 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +140 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +141 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +142 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +143 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +144 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +145 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +146 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +147 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +148 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +149 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +150 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +151 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +152 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +153 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +154 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +155 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +156 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +157 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +158 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +159 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +160 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +161 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +162 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +163 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +164 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +165 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +166 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +167 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +168 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +169 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +170 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +171 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +172 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +173 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +174 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +175 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +176 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +177 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +178 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +179 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +180 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +181 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +182 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +183 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +184 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +185 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +186 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +187 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +188 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +189 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +190 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +191 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +192 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +193 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +194 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +195 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +196 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +197 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +198 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +199 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +200 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +201 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +202 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +203 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +204 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +205 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +206 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +207 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +208 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +209 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +210 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +211 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +212 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +213 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +214 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +215 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +216 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +217 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +218 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +219 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +220 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +221 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +222 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +223 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +224 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +225 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +226 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +227 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +228 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +229 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +230 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +231 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +232 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +233 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +234 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +235 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +236 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +237 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +238 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +239 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +240 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +241 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +242 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +243 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +244 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +245 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +246 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +247 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +248 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +249 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +250 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +251 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +252 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +253 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +254 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +255 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +256 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +257 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +258 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +259 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +260 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +261 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +262 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +263 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +264 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +265 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +266 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +267 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +268 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +269 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +270 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +271 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +272 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +273 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +274 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +275 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +276 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +277 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +278 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +279 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +280 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +281 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +282 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +283 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +284 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +285 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +286 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +287 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +288 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +289 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +290 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +291 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +292 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +293 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +294 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +295 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +296 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +297 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +298 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +299 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +300 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +301 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +302 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +303 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +304 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +305 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +306 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +307 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +308 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +309 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +310 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +311 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +312 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +313 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +314 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +315 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +316 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +317 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +318 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +319 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +320 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +321 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +322 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +323 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +324 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +325 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +326 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +327 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +328 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +329 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +330 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +331 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +332 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +333 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +334 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +335 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +336 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +337 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +338 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +339 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +340 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +341 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +342 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +343 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +344 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +345 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +346 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +347 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +348 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +349 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +350 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +351 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +352 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +353 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +354 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +355 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +356 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +357 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +358 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +359 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +360 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +361 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +362 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +363 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +364 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +365 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +366 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +367 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +368 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +369 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +370 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +371 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +372 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +373 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +374 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +375 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +376 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +377 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +378 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +379 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +380 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +381 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +382 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +383 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +384 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +385 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +386 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +387 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +388 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +389 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +390 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +391 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +392 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +393 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +394 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +395 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +396 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +397 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +398 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +399 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +400 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +401 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +402 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +403 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +404 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +405 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +406 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +407 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +408 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +409 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +410 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +411 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +412 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +413 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +414 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +415 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +416 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +417 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +418 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +419 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +420 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +421 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +422 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +423 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +424 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +425 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +426 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +427 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +428 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +429 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +430 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +431 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +432 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +433 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +434 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +435 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +436 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +437 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +438 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +439 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +440 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +441 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +442 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +443 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +444 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +445 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +446 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +447 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +448 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +449 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +450 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +451 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +452 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +453 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +454 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +455 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +456 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +457 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +458 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +459 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +460 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +461 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +462 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +463 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +464 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +465 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +466 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +467 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +468 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +469 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +470 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +471 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +472 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +473 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +474 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +475 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +476 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +477 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +478 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +479 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +480 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +481 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +482 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +483 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +484 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +485 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +486 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +487 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +488 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +489 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +490 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +491 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +492 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +493 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +494 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +495 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +496 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +497 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +498 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +499 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +500 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +501 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +502 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +503 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +504 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +505 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +506 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +507 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +508 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +509 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +510 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +511 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +512 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +513 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +514 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +515 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +516 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +517 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +518 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +519 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +520 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +521 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +522 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +523 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +524 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +525 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +526 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +527 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +528 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +529 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +530 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +531 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +532 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +533 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +534 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +535 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +536 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +537 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +538 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +539 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +540 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +541 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +542 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +543 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +544 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +545 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +546 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +547 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +548 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +549 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +550 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +551 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +552 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +553 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +554 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +555 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +556 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +557 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +558 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +559 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +560 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +561 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +562 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +563 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +564 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +565 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +566 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +567 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +568 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +569 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +570 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +571 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +572 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +573 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +574 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +575 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +576 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +577 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +578 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +579 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +580 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +581 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +582 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +583 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +584 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +585 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +586 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +587 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +588 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +589 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +590 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +591 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +592 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +593 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +594 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +595 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +596 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +597 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +598 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +599 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +600 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +601 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +602 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +603 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +604 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +605 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +606 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +607 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +608 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +609 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +610 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +611 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +612 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +613 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +614 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +615 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +616 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +617 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +618 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +619 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +620 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +621 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +622 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +623 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +624 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +625 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +626 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +627 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +628 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +629 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +630 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +631 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +632 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +633 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +634 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +635 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +636 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +637 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +638 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +639 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +640 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +641 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +642 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +643 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +644 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +645 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +646 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +647 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +648 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +649 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +650 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +651 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +652 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +653 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +654 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +655 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +656 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +657 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +658 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +659 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +660 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +661 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +662 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +663 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +664 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +665 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +666 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +667 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +668 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +669 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +670 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +671 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +672 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +673 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +674 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +675 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +676 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +677 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +678 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +679 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +680 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +681 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +682 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +683 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +684 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +685 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +686 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +687 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +688 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +689 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +690 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +691 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +692 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +693 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +694 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +695 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +696 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +697 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +698 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +699 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +700 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +701 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +702 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +703 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +704 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +705 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +706 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +707 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +708 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +709 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +710 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +711 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +712 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +713 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +714 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +715 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +716 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +717 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +718 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +719 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +720 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +721 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +722 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +723 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +724 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +725 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +726 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +727 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +728 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +729 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +730 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +731 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +732 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +733 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +734 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +735 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +736 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +737 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +738 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +739 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +740 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +741 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +742 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +743 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +744 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +745 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +746 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +747 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +748 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +749 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +750 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +751 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +752 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +753 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +754 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +755 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +756 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +757 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +758 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +759 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +760 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +761 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +762 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +763 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +764 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +765 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +766 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +767 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +768 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +769 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +770 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +771 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +772 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +773 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +774 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +775 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +776 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +777 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +778 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +779 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +780 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +781 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +782 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +783 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +784 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +785 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +786 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +787 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +788 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +789 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +790 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +791 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +792 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +793 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +794 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +795 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +796 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +797 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +798 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +799 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +800 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +801 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +802 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +803 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +804 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +805 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +806 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +807 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +808 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +809 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +810 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +811 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +812 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +813 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +814 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +815 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +816 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +817 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +818 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +819 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +820 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +821 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +822 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +823 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +824 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +825 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +826 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +827 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +828 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +829 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +830 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +831 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +832 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +833 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +834 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +835 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +836 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +837 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +838 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +839 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +840 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +841 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +842 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +843 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +844 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +845 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +846 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +847 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +848 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +849 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +850 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +851 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +852 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +853 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +854 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +855 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +856 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +857 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +858 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +859 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +860 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +861 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +862 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +863 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +864 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +865 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +866 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +867 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +868 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +869 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +870 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +871 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +872 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +873 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +874 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +875 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +876 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +877 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +878 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +879 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +880 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +881 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +882 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +883 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +884 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +885 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +886 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +887 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +888 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +889 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +890 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +891 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +892 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +893 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +894 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +895 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +896 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +897 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +898 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +899 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +900 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +901 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +902 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +903 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +904 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +905 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +906 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +907 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +908 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +909 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +910 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +911 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +912 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +913 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +914 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +915 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +916 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +917 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +918 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +919 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +920 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +921 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +922 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +923 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +924 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +925 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +926 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +927 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +928 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +929 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +930 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +931 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +932 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +933 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +934 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +935 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +936 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +937 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +938 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +939 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +940 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +941 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +942 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +943 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +944 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +945 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +946 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +947 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +948 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +949 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +950 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +951 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +952 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +953 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +954 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +955 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +956 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +957 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +958 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +959 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +960 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +961 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +962 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +963 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +964 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +965 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +966 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +967 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +968 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +969 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +970 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +971 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +972 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +973 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +974 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +975 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +976 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +977 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +978 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +979 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +980 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +981 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +982 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +983 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +984 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +985 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +986 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +987 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +988 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +989 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +990 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +991 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +992 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +993 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +994 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +995 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +996 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +997 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +998 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +999 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1000 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1001 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1002 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1003 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1004 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1005 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1006 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1007 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1008 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1009 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1010 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1011 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1012 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1013 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1014 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1015 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1016 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1017 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1018 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1019 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1020 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1021 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1022 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1023 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1024 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1025 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1026 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1027 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1028 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1029 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1030 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1031 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1032 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1033 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1034 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1035 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1036 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1037 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1038 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1039 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1040 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1041 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1042 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1043 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1044 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1045 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1046 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1047 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1048 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1049 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1050 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1051 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1052 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1053 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1054 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1055 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1056 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1057 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1058 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1059 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1060 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1061 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1062 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1063 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1064 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1065 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1066 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1067 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1068 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1069 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1070 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1071 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1072 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1073 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1074 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1075 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1076 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1077 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1078 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1079 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1080 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1081 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1082 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1083 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1084 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1085 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1086 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1087 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1088 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1089 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1090 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1091 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1092 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1093 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1094 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1095 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1096 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1097 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1098 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1099 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1100 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1101 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1102 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1103 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1104 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1105 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1106 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1107 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1108 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1109 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1110 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1111 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1112 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1113 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1114 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1115 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1116 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1117 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1118 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1119 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1120 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1121 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1122 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1123 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1124 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1125 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1126 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1127 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1128 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1129 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1130 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1131 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1132 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1133 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1134 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1135 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1136 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1137 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1138 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1139 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1140 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1141 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1142 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1143 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1144 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1145 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1146 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1147 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1148 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1149 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1150 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1151 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1152 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1153 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1154 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1155 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1156 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1157 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1158 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1159 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1160 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1161 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1162 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1163 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1164 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1165 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1166 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1167 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1168 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1169 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1170 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1171 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1172 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1173 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1174 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1175 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1176 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1177 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1178 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1179 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1180 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1181 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1182 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1183 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1184 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1185 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1186 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1187 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1188 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1189 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1190 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1191 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1192 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1193 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1194 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1195 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1196 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1197 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1198 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1199 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1200 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1201 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1202 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1203 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1204 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1205 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1206 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1207 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1208 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1209 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1210 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1211 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1212 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1213 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1214 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1215 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1216 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1217 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1218 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1219 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1220 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1221 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1222 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1223 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1224 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1225 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1226 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1227 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1228 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1229 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1230 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1231 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1232 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1233 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1234 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1235 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1236 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1237 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1238 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1239 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1240 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1241 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1242 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1243 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1244 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1245 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1246 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1247 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1248 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1249 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1250 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1251 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1252 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1253 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1254 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1255 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1256 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1257 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1258 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1259 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1260 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1261 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1262 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1263 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1264 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1265 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1266 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1267 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1268 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1269 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1270 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1271 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1272 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1273 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1274 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1275 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1276 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1277 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1278 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1279 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1280 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1281 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1282 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1283 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1284 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1285 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1286 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1287 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1288 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1289 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1290 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1291 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1292 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1293 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1294 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1295 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1296 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1297 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1298 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1299 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1300 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1301 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1302 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1303 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1304 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1305 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1306 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1307 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1308 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1309 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1310 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1311 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1312 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1313 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1314 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1315 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1316 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1317 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1318 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1319 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1320 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1321 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1322 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1323 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1324 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1325 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1326 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1327 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1328 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1329 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1330 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1331 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1332 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1333 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1334 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1335 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1336 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1337 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1338 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1339 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1340 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1341 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1342 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1343 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1344 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1345 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1346 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1347 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1348 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1349 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1350 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1351 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1352 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1353 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1354 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1355 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1356 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1357 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1358 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1359 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1360 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1361 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1362 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1363 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1364 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1365 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1366 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1367 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1368 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1369 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1370 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1371 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1372 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1373 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1374 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1375 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1376 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1377 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1378 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1379 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1380 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1381 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1382 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1383 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1384 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1385 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1386 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1387 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1388 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1389 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1390 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1391 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1392 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1393 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1394 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1395 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1396 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1397 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1398 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1399 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1400 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1401 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1402 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1403 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1404 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1405 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1406 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1407 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1408 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1409 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1410 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1411 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1412 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1413 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1414 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1415 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1416 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1417 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1418 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1419 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1420 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1421 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1422 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1423 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1424 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1425 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1426 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1427 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1428 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1429 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1430 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1431 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1432 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1433 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1434 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1435 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1436 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1437 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1438 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1439 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1440 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1441 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1442 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1443 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1444 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1445 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1446 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1447 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1448 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1449 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1450 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1451 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1452 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1453 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1454 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1455 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1456 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1457 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1458 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1459 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1460 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1461 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1462 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1463 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1464 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1465 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1466 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1467 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1468 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1469 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1470 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1471 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1472 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1473 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1474 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1475 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1476 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1477 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1478 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1479 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1480 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1481 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1482 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1483 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1484 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1485 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1486 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1487 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1488 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1489 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1490 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1491 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1492 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1493 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1494 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1495 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1496 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1497 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1498 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1499 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1500 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1501 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1502 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1503 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1504 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1505 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1506 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1507 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1508 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1509 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1510 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1511 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1512 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1513 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1514 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1515 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1516 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1517 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1518 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1519 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1520 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1521 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1522 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1523 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1524 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1525 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1526 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1527 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1528 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1529 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1530 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1531 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1532 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1533 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1534 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1535 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1536 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1537 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1538 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1539 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1540 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1541 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1542 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1543 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1544 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1545 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1546 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1547 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1548 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1549 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1550 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1551 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1552 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1553 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1554 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1555 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1556 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1557 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1558 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1559 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1560 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1561 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1562 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1563 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1564 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1565 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1566 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1567 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1568 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1569 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1570 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1571 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1572 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1573 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1574 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1575 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1576 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1577 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1578 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1579 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1580 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1581 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1582 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1583 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1584 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1585 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1586 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1587 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1588 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1589 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1590 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1591 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1592 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1593 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1594 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1595 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1596 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1597 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1598 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1599 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1600 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1601 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1602 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1603 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1604 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1605 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1606 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1607 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1608 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1609 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1610 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1611 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1612 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1613 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1614 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1615 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1616 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1617 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1618 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1619 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1620 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1621 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1622 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1623 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1624 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1625 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1626 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1627 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1628 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1629 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1630 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1631 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1632 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1633 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1634 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1635 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1636 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1637 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1638 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1639 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1640 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1641 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1642 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1643 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1644 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1645 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1646 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1647 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1648 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1649 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1650 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1651 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1652 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1653 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1654 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1655 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1656 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1657 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1658 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1659 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1660 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1661 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1662 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1663 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1664 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1665 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1666 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1667 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1668 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1669 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1670 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1671 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1672 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1673 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1674 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1675 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1676 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1677 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1678 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1679 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1680 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1681 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1682 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1683 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1684 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1685 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1686 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1687 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1688 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1689 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1690 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1691 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1692 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1693 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1694 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1695 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1696 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1697 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1698 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1699 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1700 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1701 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1702 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1703 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1704 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1705 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1706 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1707 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1708 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1709 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1710 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1711 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1712 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1713 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1714 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1715 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1716 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1717 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1718 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1719 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1720 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1721 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1722 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1723 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1724 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1725 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1726 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1727 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1728 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1729 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1730 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1731 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1732 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1733 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1734 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1735 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1736 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1737 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1738 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1739 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1740 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1741 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1742 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1743 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1744 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1745 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1746 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1747 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1748 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1749 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1750 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1751 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1752 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1753 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1754 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1755 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1756 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1757 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1758 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1759 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1760 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1761 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1762 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1763 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1764 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1765 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1766 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1767 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1768 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1769 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1770 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1771 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1772 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1773 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1774 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1775 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1776 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1777 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1778 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1779 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1780 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1781 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1782 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1783 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1784 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1785 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1786 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1787 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1788 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1789 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1790 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1791 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1792 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1793 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1794 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1795 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1796 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1797 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1798 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1799 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1800 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1801 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1802 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1803 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1804 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1805 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1806 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1807 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1808 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1809 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1810 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1811 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1812 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1813 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1814 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1815 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1816 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1817 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1818 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1819 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1820 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1821 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1822 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1823 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1824 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1825 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1826 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1827 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1828 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1829 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1830 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1831 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1832 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1833 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1834 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1835 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1836 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1837 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1838 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1839 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1840 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1841 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1842 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1843 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1844 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1845 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1846 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1847 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1848 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1849 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1850 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1851 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1852 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1853 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1854 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1855 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1856 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1857 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1858 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1859 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1860 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1861 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1862 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1863 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1864 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1865 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1866 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1867 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1868 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1869 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1870 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1871 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1872 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1873 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1874 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1875 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1876 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1877 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1878 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1879 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1880 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1881 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1882 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1883 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1884 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1885 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1886 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1887 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1888 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1889 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1890 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1891 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1892 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1893 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1894 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1895 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1896 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1897 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1898 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1899 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1900 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1901 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1902 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1903 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1904 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1905 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1906 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1907 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1908 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1909 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1910 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1911 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1912 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1913 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1914 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1915 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1916 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1917 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1918 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1919 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1920 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1921 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1922 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1923 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1924 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1925 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1926 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1927 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1928 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1929 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1930 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1931 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1932 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1933 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1934 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1935 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1936 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1937 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1938 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1939 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1940 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1941 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1942 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1943 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1944 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1945 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1946 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1947 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1948 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1949 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1950 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1951 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1952 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1953 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1954 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1955 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1956 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1957 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1958 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1959 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1960 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1961 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1962 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1963 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1964 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1965 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1966 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1967 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1968 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1969 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1970 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1971 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1972 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1973 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1974 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1975 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1976 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1977 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1978 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1979 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1980 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1981 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1982 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1983 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1984 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1985 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1986 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1987 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1988 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1989 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1990 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1991 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1992 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1993 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1994 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1995 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1996 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1997 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1998 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +1999 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2000 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2001 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2002 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2003 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2004 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2005 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2006 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2007 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2008 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2009 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2010 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2011 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2012 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2013 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2014 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2015 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2016 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2017 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2018 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2019 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2020 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2021 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2022 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2023 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2024 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2025 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2026 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2027 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2028 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2029 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2030 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2031 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2032 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2033 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2034 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2035 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2036 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2037 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2038 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2039 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2040 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2041 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2042 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2043 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2044 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2045 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2046 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2047 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2048 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2049 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2050 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2051 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2052 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2053 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2054 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2055 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2056 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2057 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2058 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2059 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2060 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2061 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2062 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2063 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2064 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2065 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2066 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2067 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2068 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2069 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2070 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2071 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2072 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2073 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2074 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2075 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2076 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2077 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2078 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2079 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2080 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2081 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2082 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2083 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2084 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2085 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2086 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2087 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2088 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2089 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2090 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2091 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2092 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2093 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2094 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2095 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2096 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2097 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2098 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2099 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2100 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2101 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2102 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2103 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2104 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2105 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2106 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2107 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2108 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2109 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2110 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2111 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2112 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2113 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2114 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2115 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2116 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2117 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2118 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2119 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2120 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2121 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2122 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2123 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2124 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2125 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2126 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2127 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2128 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2129 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2130 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2131 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2132 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2133 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2134 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2135 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2136 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2137 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2138 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2139 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2140 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2141 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2142 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2143 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2144 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2145 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2146 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2147 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2148 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2149 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2150 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2151 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2152 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2153 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2154 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2155 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2156 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2157 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2158 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2159 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2160 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2161 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2162 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2163 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2164 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2165 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2166 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2167 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2168 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2169 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2170 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2171 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2172 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2173 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2174 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2175 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2176 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2177 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2178 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2179 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2180 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2181 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2182 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2183 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2184 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2185 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2186 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2187 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2188 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2189 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2190 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2191 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2192 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2193 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2194 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2195 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2196 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2197 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2198 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2199 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2200 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2201 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2202 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2203 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2204 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2205 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2206 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2207 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2208 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2209 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2210 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2211 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2212 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2213 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2214 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2215 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2216 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2217 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2218 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2219 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2220 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2221 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2222 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2223 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2224 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2225 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2226 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2227 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2228 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2229 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2230 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2231 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2232 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2233 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2234 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2235 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2236 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2237 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2238 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2239 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2240 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2241 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2242 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2243 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2244 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2245 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2246 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2247 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2248 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2249 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2250 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2251 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2252 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2253 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2254 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2255 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2256 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2257 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2258 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2259 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2260 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2261 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2262 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2263 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2264 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2265 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2266 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2267 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2268 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2269 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2270 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2271 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2272 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2273 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2274 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2275 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2276 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2277 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2278 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2279 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2280 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2281 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2282 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2283 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2284 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2285 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2286 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2287 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2288 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2289 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2290 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2291 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2292 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2293 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2294 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2295 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2296 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2297 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2298 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2299 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2300 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2301 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2302 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2303 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2304 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2305 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2306 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2307 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2308 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2309 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2310 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2311 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2312 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2313 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2314 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2315 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2316 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2317 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2318 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2319 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2320 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2321 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2322 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2323 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2324 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2325 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2326 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2327 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2328 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2329 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2330 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2331 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2332 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2333 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2334 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2335 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2336 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2337 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2338 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2339 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2340 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2341 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2342 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2343 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2344 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2345 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2346 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2347 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2348 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2349 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2350 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2351 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2352 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2353 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2354 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2355 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2356 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2357 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2358 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2359 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2360 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2361 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2362 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2363 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2364 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2365 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2366 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2367 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2368 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2369 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2370 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2371 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2372 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2373 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2374 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2375 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2376 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2377 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2378 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2379 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2380 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2381 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2382 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2383 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2384 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2385 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2386 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2387 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2388 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2389 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2390 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2391 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2392 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2393 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2394 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2395 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2396 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2397 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2398 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2399 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2400 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2401 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2402 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2403 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2404 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2405 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2406 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2407 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2408 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2409 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2410 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2411 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2412 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2413 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2414 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2415 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2416 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2417 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2418 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2419 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2420 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2421 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2422 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2423 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2424 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2425 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2426 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2427 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2428 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2429 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2430 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2431 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2432 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2433 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2434 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2435 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2436 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2437 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2438 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2439 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2440 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2441 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2442 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2443 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2444 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2445 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2446 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2447 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2448 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2449 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2450 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2451 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2452 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2453 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2454 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2455 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2456 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2457 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2458 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2459 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2460 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2461 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2462 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2463 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2464 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2465 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2466 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2467 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2468 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2469 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2470 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2471 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2472 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2473 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2474 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2475 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2476 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2477 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2478 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2479 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2480 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2481 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2482 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2483 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2484 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2485 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2486 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2487 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2488 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2489 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2490 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2491 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2492 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2493 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2494 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2495 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2496 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2497 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2498 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 +2499 2 3 4 5 6.6 7.7 9 abc def ghiaaaaaa 2020-10-10 2020-10-10T11:12:59 + +-- !sql2 -- +1 2 1025 1028 + diff --git a/regression-test/data/load_p0/http_stream/test_http_stream_properties.out b/regression-test/data/load_p0/http_stream/test_http_stream_properties.out new file mode 100644 index 00000000000000..93bde743d03850 --- /dev/null +++ b/regression-test/data/load_p0/http_stream/test_http_stream_properties.out @@ -0,0 +1,23 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql_squence -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +49 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +57 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + diff --git a/regression-test/data/load_p0/http_stream/test_stream_load1.csv b/regression-test/data/load_p0/http_stream/test_stream_load1.csv new file mode 100644 index 00000000000000..b86bcfa176fe18 --- /dev/null +++ b/regression-test/data/load_p0/http_stream/test_stream_load1.csv @@ -0,0 +1,2 @@ +5,e +6,f diff --git a/regression-test/data/load_p0/http_stream/test_stream_load2.csv b/regression-test/data/load_p0/http_stream/test_stream_load2.csv new file mode 100644 index 00000000000000..93157e3d6ddd53 --- /dev/null +++ b/regression-test/data/load_p0/http_stream/test_stream_load2.csv @@ -0,0 +1,2 @@ +70|7|e +80|8|f diff --git a/regression-test/data/load_p0/http_stream/test_stream_load3.csv b/regression-test/data/load_p0/http_stream/test_stream_load3.csv new file mode 100644 index 00000000000000..f257be1566f9e9 --- /dev/null +++ b/regression-test/data/load_p0/http_stream/test_stream_load3.csv @@ -0,0 +1,6 @@ +10,a,10 +11,a,11 +1a,a,11 +12,a +20,b,21 +101,a,101 \ No newline at end of file diff --git a/regression-test/data/load_p0/stream_load/basic_array_data.csv b/regression-test/data/load_p0/stream_load/basic_array_data.csv new file mode 100644 index 00000000000000..d25ef6489bf672 --- /dev/null +++ b/regression-test/data/load_p0/stream_load/basic_array_data.csv @@ -0,0 +1,20 @@ +55|[1, 1, 1, 1, 1, 1]|[65, 65, 65, 65, 65, 65]|[18805, 18805, 18805, 18805, 18805, 18805]|[229725878, 229725878, 229725878, 229725878, 229725878, 229725878]|[2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325]|[5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371]|[12354.624, 12354.624, 12354.624, 12354.624, 12354.624]|[1697579881.947477, 1697579881.947477, 1697579881.947477, 1697579881.947477, 1697579881.947477, 1697579881.947477]|[20409908, 20409908, 20409908, 20409908, 20409908, 20409908]|[-69625379, -69625379, -69625379, -69625379, -69625379, -69625379]|[2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09]|[2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54]|[2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19]|[2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52]|['G', 'G', 'G', 'G', 'G', 'G']|['a', 'a', 'a', 'a', 'a', 'a']|['BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7'] +84|[0, 0, 0, 0, 0, 0]|[-10, -10, -10, -10, -10, -10]|[9493, 9493, 9493, 9493, 9493, 9493]|[-547874696, -547874696, -547874696, -547874696, -547874696, -547874696]|[-115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756]|[4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085]|[13718.372, 13718.372, 13718.372, 13718.372, 13718.372]|[-978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697]|[-88516589, -88516589, -88516589, -88516589, -88516589, -88516589]|[-62683124, -62683124, -62683124, -62683124, -62683124, -62683124]|[2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26]|[2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43]|[2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25]|[2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14]|['J', 'J', 'J', 'J', 'J', 'J']|['a', 'a', 'a', 'a', 'a', 'a']|['yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq'] +48|[0, 0, 0, 0, 0, 0]|[88, 88, 88, 88, 88, 88]|[-18899, -18899, -18899, -18899, -18899, -18899]|[1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640]|[-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312]|[7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662]|[20562.791, 20562.791, 20562.791, 20562.791, 20562.791]|[-1597994654.903396, -1597994654.903396, -1597994654.903396, -1597994654.903396, -1597994654.903396, -1597994654.903396]|[91765768, 91765768, 91765768, 91765768, 91765768, 91765768]|[-47488138, -47488138, -47488138, -47488138, -47488138, -47488138]|[2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12]|[2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24]|[2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16]|[2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07]|['u', 'u', 'u', 'u', 'u', 'u']|['P', 'P', 'P', 'P', 'P', 'P']|['wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv'] +44|[0, 0, 0, 0, 0, 0]|[61, 61, 61, 61, 61, 61]|[-23419, -23419, -23419, -23419, -23419, -23419]|[378600280, 378600280, 378600280, 378600280, 378600280, 378600280]|[6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679]|[-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852]|[-20151.432, -20151.432, -20151.432, -20151.432, -20151.432]|[-1248229001.218229, -1248229001.218229, -1248229001.218229, -1248229001.218229, -1248229001.218229, -1248229001.218229]|[-45191385, -45191385, -45191385, -45191385, -45191385, -45191385]|[-27910227, -27910227, -27910227, -27910227, -27910227, -27910227]|[2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13]|[2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32]|[2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20]|[2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00]|['4', '4', '4', '4', '4', '4']|['I', 'I', 'I', 'I', 'I', 'I']|['NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1'] +4|[0, 0, 0, 0, 0, 0]|[63, 63, 63, 63, 63, 63]|[11399, 11399, 11399, 11399, 11399, 11399]|[-583523026, -583523026, -583523026, -583523026, -583523026, -583523026]|[8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293]|[-8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906]|[23243.16, 23243.16, 23243.16, 23243.16, 23243.16]|[716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115]|[-40335903, -40335903, -40335903, -40335903, -40335903, -40335903]|[74087997, 74087997, 74087997, 74087997, 74087997, 74087997]|[2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13]|[2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27]|[2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28]|[2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47]|['z', 'z', 'z', 'z', 'z', 'z']|['1', '1', '1', '1', '1', '1']|['xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz'] +4|[0, 0, 0, 0, 0, 0]|[-67, -67, -67, -67, -67, -67]|[-30372, -30372, -30372, -30372, -30372, -30372]|[181502941, 181502941, 181502941, 181502941, 181502941, 181502941]|[-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800]|[6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842]|[21235.783, 21235.783, 21235.783, 21235.783, 21235.783]|[-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891]|[58088067, 58088067, 58088067, 58088067, 58088067, 58088067]|[-66079723, -66079723, -66079723, -66079723, -66079723, -66079723]|[2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24]|[2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12]|[2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19]|[2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12]|['G', 'G', 'G', 'G', 'G', 'G']|['a', 'a', 'a', 'a', 'a', 'a']|['tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W'] +61|[1, 1, 1, 1, 1, 1]|[121, 121, 121, 121, 121, 121]|[31806, 31806, 31806, 31806, 31806, 31806]|[-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562]|[-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717]|[4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431]|[10719.892, 10719.892, 10719.892, 10719.892, 10719.892]|[1073780599.127242, 1073780599.127242, 1073780599.127242, 1073780599.127242, 1073780599.127242, 1073780599.127242]|[78876206, 78876206, 78876206, 78876206, 78876206, 78876206]|[-77837482, -77837482, -77837482, -77837482, -77837482, -77837482]|[2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01]|[2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07]|[2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15]|[2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16]|['U', 'U', 'U', 'U', 'U', 'U']|['a', 'a', 'a', 'a', 'a', 'a']|['y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb'] +62|[0, 0, 0, 0, 0, 0]|[-126, -126, -126, -126, -126, -126]|[-3813, -3813, -3813, -3813, -3813, -3813]|[1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540]|[-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217]|[5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397]|[-17121.195, -17121.195, -17121.195, -17121.195, -17121.195]|[-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562]|[9626406, 9626406, 9626406, 9626406, 9626406, 9626406]|[-8723076, -8723076, -8723076, -8723076, -8723076, -8723076]|[2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01]|[2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20]|[2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06]|[2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10]|['B', 'B', 'B', 'B', 'B', 'B']|['a', 'a', 'a', 'a', 'a', 'a']|['8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC'] +88|[1, 1, 1, 1, 1, 1]|[23, 23, 23, 23, 23, 23]|[14833, 14833, 14833, 14833, 14833, 14833]|[2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015]|[7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930]|[-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347]|[-29817.77, -29817.77, -29817.77, -29817.77, -29817.77]|[-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042]|[44414137, 44414137, 44414137, 44414137, 44414137, 44414137]|[75659236, 75659236, 75659236, 75659236, 75659236, 75659236]|[2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26]|[2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19]|[2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27]|[2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51]|['a', 'a', 'a', 'a', 'a', 'a']|['a', 'a', 'a', 'a', 'a', 'a']|['gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh'] +27|[0, 0, 0, 0, 0, 0]|[-7, -7, -7, -7, -7, -7]|[18307, 18307, 18307, 18307, 18307, 18307]|[76399879, 76399879, 76399879, 76399879, 76399879, 76399879]|[8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699]|[2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999]|[-16678.924, -16678.924, -16678.924, -16678.924, -16678.924]|[-2053879544.844726, -2053879544.844726, -2053879544.844726, -2053879544.844726, -2053879544.844726, -2053879544.844726]|[12639967, 12639967, 12639967, 12639967, 12639967, 12639967]|[24677022, 24677022, 24677022, 24677022, 24677022, 24677022]|[2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01]|[2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55]|[2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31]|[2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44]|['d', 'd', 'd', 'd', 'd', 'd']|['a', 'a', 'a', 'a', 'a', 'a']|['dGkS', 'dGkS', 'dGkS', 'dGkS', 'dGkS', 'dGkS'] +8|[1, 1, 1, 1, 1, 1]|[-11, -11, -11, -11, -11, -11]|[-9648, -9648, -9648, -9648, -9648, -9648]|[-505356927, -505356927, -505356927, -505356927, -505356927, -505356927]|[7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037]|[1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798]|[10822.962, 10822.962, 10822.962, 10822.962, 10822.962]|[1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071]|[64879544, 64879544, 64879544, 64879544, 64879544, 64879544]|[-55896622, -55896622, -55896622, -55896622, -55896622, -55896622]|[2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08]|[2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23]|[2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16]|[2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26]|['U', 'U', 'U', 'U', 'U', 'U']|['G', 'G', 'G', 'G', 'G', 'G']|['iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ'] +76|[1, 1, 1, 1, 1, 1]|[-63, -63, -63, -63, -63, -63]|[25799, 25799, 25799, 25799, 25799, 25799]|[-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656]|[8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181]|[-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476]|[5311.188, 5311.188, 5311.188, 5311.188, 5311.188]|[173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049]|[-28850258, -28850258, -28850258, -28850258, -28850258, -28850258]|[-99202733, -99202733, -99202733, -99202733, -99202733, -99202733]|[2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12]|[2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22]|[2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11]|[2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10]|['7', '7', '7', '7', '7', '7']|['j', 'j', 'j', 'j', 'j', 'j']|['fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0'] +52|[0, 0, 0, 0, 0, 0]|[-7, -7, -7, -7, -7, -7]|[-6985, -6985, -6985, -6985, -6985, -6985]|[826683531, 826683531, 826683531, 826683531, 826683531, 826683531]|[-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361]|[4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446]|[-6490.247, -6490.247, -6490.247, -6490.247, -6490.247]|[1076976372.033826, 1076976372.033826, 1076976372.033826, 1076976372.033826, 1076976372.033826, 1076976372.033826]|[-25002204, -25002204, -25002204, -25002204, -25002204, -25002204]|[-97805693, -97805693, -97805693, -97805693, -97805693, -97805693]|[2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02]|[2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22]|[2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19]|[2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09]|['p', 'p', 'p', 'p', 'p', 'p']|['a', 'a', 'a', 'a', 'a', 'a']|['WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p'] +3|[0, 0, 0, 0, 0, 0]|[65, 65, 65, 65, 65, 65]|[-4963, -4963, -4963, -4963, -4963, -4963]|[-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954]|[-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000]|[8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376]|[-5058.13, -5058.13, -5058.13, -5058.13, -5058.13]|[1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352]|[3858273, 3858273, 3858273, 3858273, 3858273, 3858273]|[-3634150, -3634150, -3634150, -3634150, -3634150, -3634150]|[2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26]|[2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44]|[2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27]|[2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11]|['i', 'i', 'i', 'i', 'i', 'i']|['G', 'G', 'G', 'G', 'G', 'G']|['XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ'] +49|[0, 0, 0, 0, 0, 0]|[126, 126, 126, 126, 126, 126]|[31661, 31661, 31661, 31661, 31661, 31661]|[359703581, 359703581, 359703581, 359703581, 359703581, 359703581]|[-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939]|[7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391]|[-504.68152, -504.68152, -504.68152, -504.68152, -504.68152]|[2121528178.488779, 2121528178.488779, 2121528178.488779, 2121528178.488779, 2121528178.488779, 2121528178.488779]|[-99977803, -99977803, -99977803, -99977803, -99977803, -99977803]|[25652192, 25652192, 25652192, 25652192, 25652192, 25652192]|[2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25]|[2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11]|[2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28]|[2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11]|['i', 'i', 'i', 'i', 'i', 'i']|['0', '0', '0', '0', '0', '0']|['X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7'] +41|[0, 0, 0, 0, 0, 0]|[-25, -25, -25, -25, -25, -25]|[28704, 28704, 28704, 28704, 28704, 28704]|[-437867812, -437867812, -437867812, -437867812, -437867812, -437867812]|[5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079]|[462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597]|[13629.614, 13629.614, 13629.614, 13629.614, 13629.614]|[213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962]|[86704879, 86704879, 86704879, 86704879, 86704879, 86704879]|[-2662959, -2662959, -2662959, -2662959, -2662959, -2662959]|[2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04]|[2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08]|[2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13]|[2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07]|['4', '4', '4', '4', '4', '4']|['a', 'a', 'a', 'a', 'a', 'a']|['2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI'] +16|[1, 1, 1, 1, 1, 1]|[-7, -7, -7, -7, -7, -7]|[18655, 18655, 18655, 18655, 18655, 18655]|[6240356, 6240356, 6240356, 6240356, 6240356, 6240356]|[4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393]|[323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978]|[2972.2478, 2972.2478, 2972.2478, 2972.2478, 2972.2478]|[-1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008]|[-84949097, -84949097, -84949097, -84949097, -84949097, -84949097]|[29815991, 29815991, 29815991, 29815991, 29815991, 29815991]|[2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24]|[2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11]|[2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10]|[2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07]|['9', '9', '9', '9', '9', '9']|['n', 'n', 'n', 'n', 'n', 'n']|['s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX'] +16|[0, 0, 0, 0, 0, 0]|[-43, -43, -43, -43, -43, -43]|[13560, 13560, 13560, 13560, 13560, 13560]|[-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513]|[7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769]|[-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413]|[12225.427, 12225.427, 12225.427, 12225.427, 12225.427]|[-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191]|[-35959452, -35959452, -35959452, -35959452, -35959452, -35959452]|[-4256846, -4256846, -4256846, -4256846, -4256846, -4256846]|[2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24]|[2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43]|[2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05]|[2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04]|['V', 'V', 'V', 'V', 'V', 'V']|['a', 'a', 'a', 'a', 'a', 'a']|['8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj'] +17|[1, 1, 1, 1, 1, 1]|[126, 126, 126, 126, 126, 126]|[28165, 28165, 28165, 28165, 28165, 28165]|[2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721]|[-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879]|[-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876]|[-21681.223, -21681.223, -21681.223, -21681.223, -21681.223]|[95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678]|[89062729, 89062729, 89062729, 89062729, 89062729, 89062729]|[-2464406, -2464406, -2464406, -2464406, -2464406, -2464406]|[2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16]|[2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36]|[2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20]|[2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44]|['U', 'U', 'U', 'U', 'U', 'U']|['a', 'a', 'a', 'a', 'a', 'a']|['XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb'] +2|[0, 0, 0, 0, 0, 0]|[117, 117, 117, 117, 117, 117]|[-4744, -4744, -4744, -4744, -4744, -4744]|[-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961]|[-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780]|[8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567]|[-30948.857, -30948.857, -30948.857, -30948.857, -30948.857]|[804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905]|[-74019648, -74019648, -74019648, -74019648, -74019648, -74019648]|[13024168, 13024168, 13024168, 13024168, 13024168, 13024168]|[2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22]|[2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12]|[2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21]|[2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56]|['g', 'g', 'g', 'g', 'g', 'g']|['a', 'a', 'a', 'a', 'a', 'a']|['S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl'] diff --git a/regression-test/data/load_p0/stream_load/basic_array_data.csv.bz2 b/regression-test/data/load_p0/stream_load/basic_array_data.csv.bz2 new file mode 100644 index 00000000000000..7fe73b0e0d53b4 Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_array_data.csv.bz2 differ diff --git a/regression-test/data/load_p0/stream_load/basic_array_data.csv.deflate b/regression-test/data/load_p0/stream_load/basic_array_data.csv.deflate new file mode 100644 index 00000000000000..f4c5b87626a6b0 Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_array_data.csv.deflate differ diff --git a/regression-test/data/load_p0/stream_load/basic_array_data.csv.gz b/regression-test/data/load_p0/stream_load/basic_array_data.csv.gz new file mode 100644 index 00000000000000..0f88cce595b8d5 Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_array_data.csv.gz differ diff --git a/regression-test/data/load_p0/stream_load/basic_array_data.csv.lz4 b/regression-test/data/load_p0/stream_load/basic_array_data.csv.lz4 new file mode 100644 index 00000000000000..1d144ea21d5470 Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_array_data.csv.lz4 differ diff --git a/regression-test/data/load_p0/stream_load/basic_array_data.csv.lzo b/regression-test/data/load_p0/stream_load/basic_array_data.csv.lzo new file mode 100644 index 00000000000000..b92042c5e1f832 Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_array_data.csv.lzo differ diff --git a/regression-test/data/load_p0/stream_load/basic_array_data_timezone.csv b/regression-test/data/load_p0/stream_load/basic_array_data_timezone.csv new file mode 100644 index 00000000000000..a08b92360c6fe8 --- /dev/null +++ b/regression-test/data/load_p0/stream_load/basic_array_data_timezone.csv @@ -0,0 +1,20 @@ +[1, 1, 1, 1, 1, 1]|[65, 65, 65, 65, 65, 65]|[18805, 18805, 18805, 18805, 18805, 18805]|[229725878, 229725878, 229725878, 229725878, 229725878, 229725878]|[2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325]|[5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371]|[12354.624, 12354.624, 12354.624, 12354.624, 12354.624]|[1697579881.947477, 1697579881.947477, 1697579881.947477, 1697579881.947477, 1697579881.947477, 1697579881.947477]|[20409908, 20409908, 20409908, 20409908, 20409908, 20409908]|[-69625379, -69625379, -69625379, -69625379, -69625379, -69625379]|[2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09]|[2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54]|[2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19]|[2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52]|['G', 'G', 'G', 'G', 'G', 'G']|['a', 'a', 'a', 'a', 'a', 'a']|['BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7'] +[0, 0, 0, 0, 0, 0]|[-10, -10, -10, -10, -10, -10]|[9493, 9493, 9493, 9493, 9493, 9493]|[-547874696, -547874696, -547874696, -547874696, -547874696, -547874696]|[-115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756]|[4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085]|[13718.372, 13718.372, 13718.372, 13718.372, 13718.372]|[-978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697]|[-88516589, -88516589, -88516589, -88516589, -88516589, -88516589]|[-62683124, -62683124, -62683124, -62683124, -62683124, -62683124]|[2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26]|[2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43]|[2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25]|[2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14]|['J', 'J', 'J', 'J', 'J', 'J']|['a', 'a', 'a', 'a', 'a', 'a']|['yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq'] +[0, 0, 0, 0, 0, 0]|[88, 88, 88, 88, 88, 88]|[-18899, -18899, -18899, -18899, -18899, -18899]|[1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640]|[-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312]|[7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662]|[20562.791, 20562.791, 20562.791, 20562.791, 20562.791]|[-1597994654.903396, -1597994654.903396, -1597994654.903396, -1597994654.903396, -1597994654.903396, -1597994654.903396]|[91765768, 91765768, 91765768, 91765768, 91765768, 91765768]|[-47488138, -47488138, -47488138, -47488138, -47488138, -47488138]|[2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12]|[2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24]|[2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16]|[2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07]|['u', 'u', 'u', 'u', 'u', 'u']|['P', 'P', 'P', 'P', 'P', 'P']|['wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv'] +[0, 0, 0, 0, 0, 0]|[61, 61, 61, 61, 61, 61]|[-23419, -23419, -23419, -23419, -23419, -23419]|[378600280, 378600280, 378600280, 378600280, 378600280, 378600280]|[6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679]|[-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852]|[-20151.432, -20151.432, -20151.432, -20151.432, -20151.432]|[-1248229001.218229, -1248229001.218229, -1248229001.218229, -1248229001.218229, -1248229001.218229, -1248229001.218229]|[-45191385, -45191385, -45191385, -45191385, -45191385, -45191385]|[-27910227, -27910227, -27910227, -27910227, -27910227, -27910227]|[2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13]|[2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32]|[2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20]|[2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00]|['4', '4', '4', '4', '4', '4']|['I', 'I', 'I', 'I', 'I', 'I']|['NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1'] +[0, 0, 0, 0, 0, 0]|[63, 63, 63, 63, 63, 63]|[11399, 11399, 11399, 11399, 11399, 11399]|[-583523026, -583523026, -583523026, -583523026, -583523026, -583523026]|[8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293]|[-8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906]|[23243.16, 23243.16, 23243.16, 23243.16, 23243.16]|[716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115]|[-40335903, -40335903, -40335903, -40335903, -40335903, -40335903]|[74087997, 74087997, 74087997, 74087997, 74087997, 74087997]|[2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13]|[2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27]|[2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28]|[2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47]|['z', 'z', 'z', 'z', 'z', 'z']|['1', '1', '1', '1', '1', '1']|['xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz'] +[0, 0, 0, 0, 0, 0]|[-67, -67, -67, -67, -67, -67]|[-30372, -30372, -30372, -30372, -30372, -30372]|[181502941, 181502941, 181502941, 181502941, 181502941, 181502941]|[-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800]|[6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842]|[21235.783, 21235.783, 21235.783, 21235.783, 21235.783]|[-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891]|[58088067, 58088067, 58088067, 58088067, 58088067, 58088067]|[-66079723, -66079723, -66079723, -66079723, -66079723, -66079723]|[2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24]|[2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12]|[2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19]|[2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12]|['G', 'G', 'G', 'G', 'G', 'G']|['a', 'a', 'a', 'a', 'a', 'a']|['tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W'] +[1, 1, 1, 1, 1, 1]|[121, 121, 121, 121, 121, 121]|[31806, 31806, 31806, 31806, 31806, 31806]|[-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562]|[-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717]|[4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431]|[10719.892, 10719.892, 10719.892, 10719.892, 10719.892]|[1073780599.127242, 1073780599.127242, 1073780599.127242, 1073780599.127242, 1073780599.127242, 1073780599.127242]|[78876206, 78876206, 78876206, 78876206, 78876206, 78876206]|[-77837482, -77837482, -77837482, -77837482, -77837482, -77837482]|[2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01]|[2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07]|[2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15]|[2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16]|['U', 'U', 'U', 'U', 'U', 'U']|['a', 'a', 'a', 'a', 'a', 'a']|['y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb'] +[0, 0, 0, 0, 0, 0]|[-126, -126, -126, -126, -126, -126]|[-3813, -3813, -3813, -3813, -3813, -3813]|[1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540]|[-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217]|[5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397]|[-17121.195, -17121.195, -17121.195, -17121.195, -17121.195]|[-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562]|[9626406, 9626406, 9626406, 9626406, 9626406, 9626406]|[-8723076, -8723076, -8723076, -8723076, -8723076, -8723076]|[2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01]|[2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20]|[2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06]|[2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10]|['B', 'B', 'B', 'B', 'B', 'B']|['a', 'a', 'a', 'a', 'a', 'a']|['8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC'] +[1, 1, 1, 1, 1, 1]|[23, 23, 23, 23, 23, 23]|[14833, 14833, 14833, 14833, 14833, 14833]|[2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015]|[7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930]|[-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347]|[-29817.77, -29817.77, -29817.77, -29817.77, -29817.77]|[-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042]|[44414137, 44414137, 44414137, 44414137, 44414137, 44414137]|[75659236, 75659236, 75659236, 75659236, 75659236, 75659236]|[2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26]|[2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19]|[2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27]|[2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51]|['a', 'a', 'a', 'a', 'a', 'a']|['a', 'a', 'a', 'a', 'a', 'a']|['gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh'] +[0, 0, 0, 0, 0, 0]|[-7, -7, -7, -7, -7, -7]|[18307, 18307, 18307, 18307, 18307, 18307]|[76399879, 76399879, 76399879, 76399879, 76399879, 76399879]|[8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699]|[2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999]|[-16678.924, -16678.924, -16678.924, -16678.924, -16678.924]|[-2053879544.844726, -2053879544.844726, -2053879544.844726, -2053879544.844726, -2053879544.844726, -2053879544.844726]|[12639967, 12639967, 12639967, 12639967, 12639967, 12639967]|[24677022, 24677022, 24677022, 24677022, 24677022, 24677022]|[2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01]|[2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55]|[2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31]|[2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44]|['d', 'd', 'd', 'd', 'd', 'd']|['a', 'a', 'a', 'a', 'a', 'a']|['dGkS', 'dGkS', 'dGkS', 'dGkS', 'dGkS', 'dGkS'] +[1, 1, 1, 1, 1, 1]|[-11, -11, -11, -11, -11, -11]|[-9648, -9648, -9648, -9648, -9648, -9648]|[-505356927, -505356927, -505356927, -505356927, -505356927, -505356927]|[7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037]|[1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798]|[10822.962, 10822.962, 10822.962, 10822.962, 10822.962]|[1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071]|[64879544, 64879544, 64879544, 64879544, 64879544, 64879544]|[-55896622, -55896622, -55896622, -55896622, -55896622, -55896622]|[2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08]|[2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23]|[2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16]|[2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26]|['U', 'U', 'U', 'U', 'U', 'U']|['G', 'G', 'G', 'G', 'G', 'G']|['iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ'] +[1, 1, 1, 1, 1, 1]|[-63, -63, -63, -63, -63, -63]|[25799, 25799, 25799, 25799, 25799, 25799]|[-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656]|[8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181]|[-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476]|[5311.188, 5311.188, 5311.188, 5311.188, 5311.188]|[173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049]|[-28850258, -28850258, -28850258, -28850258, -28850258, -28850258]|[-99202733, -99202733, -99202733, -99202733, -99202733, -99202733]|[2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12]|[2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22]|[2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11]|[2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10]|['7', '7', '7', '7', '7', '7']|['j', 'j', 'j', 'j', 'j', 'j']|['fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0'] +[0, 0, 0, 0, 0, 0]|[-7, -7, -7, -7, -7, -7]|[-6985, -6985, -6985, -6985, -6985, -6985]|[826683531, 826683531, 826683531, 826683531, 826683531, 826683531]|[-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361]|[4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446]|[-6490.247, -6490.247, -6490.247, -6490.247, -6490.247]|[1076976372.033826, 1076976372.033826, 1076976372.033826, 1076976372.033826, 1076976372.033826, 1076976372.033826]|[-25002204, -25002204, -25002204, -25002204, -25002204, -25002204]|[-97805693, -97805693, -97805693, -97805693, -97805693, -97805693]|[2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02]|[2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22]|[2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19]|[2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09]|['p', 'p', 'p', 'p', 'p', 'p']|['a', 'a', 'a', 'a', 'a', 'a']|['WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p'] +[0, 0, 0, 0, 0, 0]|[65, 65, 65, 65, 65, 65]|[-4963, -4963, -4963, -4963, -4963, -4963]|[-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954]|[-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000]|[8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376]|[-5058.13, -5058.13, -5058.13, -5058.13, -5058.13]|[1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352]|[3858273, 3858273, 3858273, 3858273, 3858273, 3858273]|[-3634150, -3634150, -3634150, -3634150, -3634150, -3634150]|[2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26]|[2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44]|[2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27]|[2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11]|['i', 'i', 'i', 'i', 'i', 'i']|['G', 'G', 'G', 'G', 'G', 'G']|['XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ'] +[0, 0, 0, 0, 0, 0]|[126, 126, 126, 126, 126, 126]|[31661, 31661, 31661, 31661, 31661, 31661]|[359703581, 359703581, 359703581, 359703581, 359703581, 359703581]|[-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939]|[7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391]|[-504.68152, -504.68152, -504.68152, -504.68152, -504.68152]|[2121528178.488779, 2121528178.488779, 2121528178.488779, 2121528178.488779, 2121528178.488779, 2121528178.488779]|[-99977803, -99977803, -99977803, -99977803, -99977803, -99977803]|[25652192, 25652192, 25652192, 25652192, 25652192, 25652192]|[2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25]|[2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11]|[2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28]|[2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11]|['i', 'i', 'i', 'i', 'i', 'i']|['0', '0', '0', '0', '0', '0']|['X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7'] +[0, 0, 0, 0, 0, 0]|[-25, -25, -25, -25, -25, -25]|[28704, 28704, 28704, 28704, 28704, 28704]|[-437867812, -437867812, -437867812, -437867812, -437867812, -437867812]|[5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079]|[462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597]|[13629.614, 13629.614, 13629.614, 13629.614, 13629.614]|[213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962]|[86704879, 86704879, 86704879, 86704879, 86704879, 86704879]|[-2662959, -2662959, -2662959, -2662959, -2662959, -2662959]|[2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04]|[2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08]|[2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13]|[2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07]|['4', '4', '4', '4', '4', '4']|['a', 'a', 'a', 'a', 'a', 'a']|['2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI'] +[1, 1, 1, 1, 1, 1]|[-7, -7, -7, -7, -7, -7]|[18655, 18655, 18655, 18655, 18655, 18655]|[6240356, 6240356, 6240356, 6240356, 6240356, 6240356]|[4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393]|[323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978]|[2972.2478, 2972.2478, 2972.2478, 2972.2478, 2972.2478]|[-1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008]|[-84949097, -84949097, -84949097, -84949097, -84949097, -84949097]|[29815991, 29815991, 29815991, 29815991, 29815991, 29815991]|[2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24]|[2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11]|[2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10]|[2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07]|['9', '9', '9', '9', '9', '9']|['n', 'n', 'n', 'n', 'n', 'n']|['s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX'] +[0, 0, 0, 0, 0, 0]|[-43, -43, -43, -43, -43, -43]|[13560, 13560, 13560, 13560, 13560, 13560]|[-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513]|[7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769]|[-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413]|[12225.427, 12225.427, 12225.427, 12225.427, 12225.427]|[-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191]|[-35959452, -35959452, -35959452, -35959452, -35959452, -35959452]|[-4256846, -4256846, -4256846, -4256846, -4256846, -4256846]|[2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24]|[2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43]|[2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05]|[2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04]|['V', 'V', 'V', 'V', 'V', 'V']|['a', 'a', 'a', 'a', 'a', 'a']|['8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj'] +[1, 1, 1, 1, 1, 1]|[126, 126, 126, 126, 126, 126]|[28165, 28165, 28165, 28165, 28165, 28165]|[2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721]|[-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879]|[-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876]|[-21681.223, -21681.223, -21681.223, -21681.223, -21681.223]|[95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678]|[89062729, 89062729, 89062729, 89062729, 89062729, 89062729]|[-2464406, -2464406, -2464406, -2464406, -2464406, -2464406]|[2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16]|[2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36]|[2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20]|[2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44]|['U', 'U', 'U', 'U', 'U', 'U']|['a', 'a', 'a', 'a', 'a', 'a']|['XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb'] +[0, 0, 0, 0, 0, 0]|[117, 117, 117, 117, 117, 117]|[-4744, -4744, -4744, -4744, -4744, -4744]|[-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961]|[-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780]|[8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567]|[-30948.857, -30948.857, -30948.857, -30948.857, -30948.857]|[804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905]|[-74019648, -74019648, -74019648, -74019648, -74019648, -74019648]|[13024168, 13024168, 13024168, 13024168, 13024168, 13024168]|[2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22]|[2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12]|[2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21]|[2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56]|['g', 'g', 'g', 'g', 'g', 'g']|['a', 'a', 'a', 'a', 'a', 'a']|['S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl'] diff --git a/regression-test/data/load_p0/stream_load/basic_array_data_with_errors.csv b/regression-test/data/load_p0/stream_load/basic_array_data_with_errors.csv new file mode 100644 index 00000000000000..1f732754edd677 --- /dev/null +++ b/regression-test/data/load_p0/stream_load/basic_array_data_with_errors.csv @@ -0,0 +1,20 @@ +55|[, 1, 1, 1, 1, 1]|[65, 65, 65, 65, 65, 65]|[18805, 18805, 18805, 18805, 18805, 18805]|[229725878, 229725878, 229725878, 229725878, 229725878, 229725878]|[2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325]|[5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371]|[12354.624, 12354.624, 12354.624, 12354.624, 12354.624]|[1697579881.947477, 1697579881.947477, 1697579881.947477, 1697579881.947477, 1697579881.947477, 1697579881.947477]|[20409908, 20409908, 20409908, 20409908, 20409908, 20409908]|[-69625379, -69625379, -69625379, -69625379, -69625379, -69625379]|[2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09]|[2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54]|[2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19]|[2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52]|['G', 'G', 'G', 'G', 'G', 'G']|['a', 'a', 'a', 'a', 'a', 'a']|['BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7', 'BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7'] +84|[0, 0, 0, 0, 0, 0]|[-10, -10, -10, -10, -10, -10|[9493, 9493, 9493, 9493, 9493, 9493]|[-547874696, -547874696, -547874696, -547874696, -547874696, -547874696]|[-115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756]|[4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085]|[13718.372, 13718.372, 13718.372, 13718.372, 13718.372]|[-978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697]|[-88516589, -88516589, -88516589, -88516589, -88516589, -88516589]|[-62683124, -62683124, -62683124, -62683124, -62683124, -62683124]|[2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26]|[2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43]|[2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25]|[2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14]|['J', 'J', 'J', 'J', 'J', 'J']|['', '', '', '', '', '']|['yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq', 'yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq'] +48|[0, 0, 0, 0, 0, 2]|[88, 88, 88, 88, 88, 88]|[-18899, -18899, -18899, -18899, -18899, -18899]|[1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640]|[-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312]|[7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662]|[20562.791, 20562.791, 20562.791, 20562.791, 20562.791]|[-1597994654.903396, -1597994654.903396, -1597994654.903396, -1597994654.903396, -1597994654.903396, -1597994654.903396]|[91765768, 91765768, 91765768, 91765768, 91765768, 91765768]|[-47488138, -47488138, -47488138, -47488138, -47488138, -47488138]|[2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12]|[2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24]|[2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16]|[2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07]|['u', 'u', 'u', 'u', 'u', 'u']|['P', 'P', 'P', 'P', 'P', 'P']|['wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv', 'wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv'] +44|[0, 0, 0, 0, 0, 0]|[61, 61, 61, 61, 61, 61999999999999999999]|[-23419, -23419, -23419, -23419, -23419, -23419]|[378600280, 378600280, 378600280, 378600280, 378600280, 378600280]|[6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679]|[-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852]|[-20151.432, -20151.432, -20151.432, -20151.432, -20151.432]|[-1248229001.218229, -1248229001.218229, -1248229001.218229, -1248229001.218229, -1248229001.218229, -1248229001.218229]|[-45191385, -45191385, -45191385, -45191385, -45191385, -45191385]|[-27910227, -27910227, -27910227, -27910227, -27910227, -27910227]|[2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13]|[2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32]|[2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20]|[2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00]|['4', '4', '4', '4', '4', '4']|['I', 'I', 'I', 'I', 'I', 'I']|['NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1', 'NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1'] +4|[0, 0, 0, 0, 0, 0]|[63, 63, 63, 63, 63, 63]|[11399, 11399, 11399, 11399, 11399, 11399]|[-583523026, -583523026, -583523026, -583523026, -583523026, -583523026]|[8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293]|[-8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906]|[23243.16, 23243.16, 23243.16, 23243.16, 23243.16]|[716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115]|[-40335903, -40335903, -40335903, -40335903, -40335903, -40335903]|[74087997, 74087997, 74087997, 74087997, 74087997, 74087997]|[2023-08-13 12:00:00, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13]|[2023-08-06, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27:111, 2023-08-06 19:03:27, 2023-08-06 19:03:27]|[2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28]|[2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47]|['z', 'z', 'z', 'z', 'z', 'z']|['1', '1', '1', '1', '1', '1']|['xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz', 'xdc5RcpnOAqeK6Hmz'] +4|[0, 0, 0, 0, 0, 0]|[-67, -67, -67, -67, -67, -67]|[-30372, -30372, -30372, -30372, -30372, -30372]|[181502941, 181502941, 181502941, 181502941, 181502941, 181502941]|[-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800]|[6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842]|[21235.783, 21235.783, 21235.783, 21235.783, 21235.783]|[-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891]|[58088067, 58088067, 58088067, 58088067, 5808806.7, 58088067]|[-66079723, -66079723, -66079723, -66079723, -66079723, -66079723.0]|[2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24]|[2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12]|[2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19]|[2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12]|['G', 'G', 'G', 'G', 'G', 'G']|['', '', '', '', '', '']|['tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W', 'tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W'] +61|[1, 1, 1, 1, 1, 1]|[121, 121, 121, 121, 121, 121]|[31806, 31806, 31806, 31806, 31806, 31806]|[-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562]|[-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717]|[4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431]|[10719.892, 10719.892, 10719.892, 10719.892, 10719.892]|[1073780599.127242, 1073780599.127242, 1073780599.127242, 1073780599.127242, 1073780599.127242, 1073780599.127242]|[78876206, 78876206, 78876206, 78876206, 78876206, 78876206]|[-77837482, -77837482, -77i837482, -77837482, -77837482, -77837482]|[2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01]|[2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07]|[2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15]|[2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16]|['U', 'U', 'U', 'U', 'U', 'U']|['', '', '', '', '', '']|['y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb', 'y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb'] +62|[0, 0, 0, 0, 0, 0]|[-126, -126, -126, -126, -126, -126]|[-3813, -3813, -3813, -3813, -3813, -3813]|[1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540]|[-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217]|[5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397]|[-17121.195, -17121.195, -17121.195, -17121.195, -17121.195]|[-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562]|[9626406, 9626406, 9626406, 9626406, 9626406, 9626406]|[2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01]|[2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20]|[2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06]|[2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10]|['B', 'B', 'B', 'B', 'B', 'B']|['', '', '', '', '', '']|['8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC', '8aRmFSUBC'] +88|[1, 1, 1, 1, 1, 1]|[23, 23, 23, 23, 23, 23]|[14833, 14833, 14833, 14833, 14833, 14833]|[2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015]|[7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930]|[-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347]|[-29817.77, -29817.77, -29817.77, -29817.77, -29817.77]|[-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042]|[44414137, 44414137, 44414137, 44414137, 44414137, 44414137]||[75659236, 75659236, 75659236, 75659236, 75659236, 75659236]|[2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26]|[2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19]|[2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27]|[2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51]|['a', 'a', 'a', 'a', 'a', 'a']|['', '', '', '', '', '']|['gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh', 'gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh'] +27|[0, 0, 0, 0, 0, 0]|[-7, -7, -7, -7, -7, -7]|[18307, 18307, 18307, 18307, 18307, 18307]|[76399879, 76399879, 76399879, 76399879, 76399879, 76399879]|[8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699]|[2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999]|[-16678.924, -16678.924, -16678.924, -16678.924, -16678.924]|[-2053879544.844726, -2053879544.844726, -2053879544.844726, -2053879544.844726, -2053879544.844726, -2053879544.844726]|[12639967, 12639967, 12639967, 12639967, 12639967, 12639967]|[24677022, 24677022, 24677022, 24677022, 24677022, 24677022]|[2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01,, 2023-08-01]|[2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55]|[2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31]|[2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44]|['d', 'd', 'd', 'd', 'd', 'd']|['', '', '', '', '', '']|['dGkS', 'dGkS', 'dGkS', 'dGkS', 'dGkS', 'dGkS'] +8|[1, 1, 1, 1, 1, 1]|[-11, -11, -11, -11, -11, -11]|[-9648, -9648, -9648, -9648, -9648, -9648]|[-505356927, -505356927, -505356927, -505356927, -505356927, -505356927]|[7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037]|[1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798]|[10822.962, 10822.962, 10822.962, 10822.962, 10822.962]|[1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071]|[64879544, 64879544, 64879544, 64879544, 64879544, 64879544]|[-55896622, -55896622, -55896622, -55896622, -55896622, -55896622]|[2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08]|[2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23]|[2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16]|[2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26]|['U', 'U', 'U', 'U', 'U', 'U']|['G', 'G', 'G', 'G', 'G', 'G']|['iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ', 'iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ'] +76|[1, 1, 1, 1, 1, 1]|[-63, -63, -63, -63, -63, -63]|[25799, 25799, 25799, 25799, 25799, 25799]|[-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656]|[8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181]|[-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476]|[5311.188, 5311.188, 5311.188, 5311.188, 5311.188]|[173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049]|[-28850258, -28850258, -28850258, -28850258, -28850258, -28850258]|[-99202733, -99202733, -99202733, -99202733, -99202733, -99202733]|[2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12]|[2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22]|[2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11]|[2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10]|['7', '7', '7', '7', '7', '7']|['j', 'j', 'j', 'j', 'j', 'j']|['fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0', 'fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0'] +52|[0, 0, 0, 0, 0, 0]|[-7, -7, -7, -7, -7, -7]|[-6985, -6985, -6985, -6985, -6985, -6985]|[826683531, 826683531, 826683531, 826683531, 826683531, 826683531]|[-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361]|[4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446]|[-6490.247, -6490.247, -6490.247, -6490.247, -6490.247]|[1076976372.033826, 1076976372.033826, 1076976372.033826, 1076976372.033826, 1076976372.033826, 1076976372.033826]|[-25002204, -25002204, -25002204, -25002204, -25002204, -25002204]|[-97805693, -97805693, -97805693, -97805693, -97805693, -97805693]|[2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02]|[2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22]|[2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19]|[2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09]|['p', 'p', 'p', 'p', 'p', 'p']|['', '', '', '', '', '']|['WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p', 'WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p'] +3|[0, 0, 0, 0, 0, 0]|[65, 65, 65, 65, 65, 65]|[-4963, -4963, -4963, -4963, -4963, -4963]|[-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954]|[-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000]|[8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376]|[-5058.13, -5058.13, -5058.13, -5058.13, -5058.13]|[1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352]|[3858273, 3858273, 3858273, 3858273, 3858273, 3858273]|[-3634150, -3634150, -3634150, -3634150, -3634150, -3634150]|[2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26]|[2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44]|[2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27]|[2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11]|['i', 'i', 'i', 'i', 'i', 'i']|['G', 'G', 'G', 'G', 'G', 'G']|['XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ', 'XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ'] +49|[0, 0, 0, 0, 0, 0]|[126, 126, 126, 126, 126, 126]|[31661, 31661, 31661, 31661, 31661, 31661]|[359703581, 359703581, 359703581, 359703581, 359703581, 359703581]|[-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939]|[7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391]|[-504.68152, -504.68152, -504.68152, -504.68152, -504.68152]|[2121528178.488779, 2121528178.488779, 2121528178.488779, 2121528178.488779, 2121528178.488779, 2121528178.488779]|[-99977803, -99977803, -99977803, -99977803, -99977803, -99977803]|[25652192, 25652192, 25652192, 25652192, 25652192, 25652192]|[2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25]|[2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11]|[2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28]|[2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11]|['i', 'i', 'i', 'i', 'i', 'i']|['0', '0', '0', '0', '0', '0']|['X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7', 'X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7'] +41|[0, 0, 0, 0, 0, 0]|[-25, -25, -25, -25, -25, -25]|[28704, 28704, 28704, 28704, 28704, 28704]|[-437867812, -437867812, -437867812, -437867812, -437867812, -437867812]|[5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079]|[462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597]|[13629.614, 13629.614, 13629.614, 13629.614, 13629.614]|[213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962]|[86704879, 86704879, 86704879, 86704879, 86704879, 86704879]|[-2662959, -2662959, -2662959, -2662959, -2662959, -2662959]|[2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04]|[2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08]|[2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13]|[2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07]|['4', '4', '4', '4', '4', '4']|['', '', '', '', '', '']|['2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI', '2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI'] +16|[1, 1, 1, 1, 1, 1]|[-7, -7, -7, -7, -7, -7]|[18655, 18655, 18655, 18655, 18655, 18655]|[6240356, 6240356, 6240356, 6240356, 6240356, 6240356]|[4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393]|[323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978]|[2972.2478, 2972.2478, 2972.2478, 2972.2478, 2972.2478]|[-1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008]|[-84949097, -84949097, -84949097, -84949097, -84949097, -84949097]|[29815991, 29815991, 29815991, 29815991, 29815991, 29815991]|[2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24]|[2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11]|[2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10]|[2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07]|['9', '9', '9', '9', '9', '9']|['n', 'n', 'n', 'n', 'n', 'n']|['s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX', 's093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX'] +16|[0, 0, 0, 0, 0, 0]|[-43, -43, -43, -43, -43, -43]|[13560, 13560, 13560, 13560, 13560, 13560]|[-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513]|[7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769]|[-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413]|[12225.427, 12225.427, 12225.427, 12225.427, 12225.427]|[-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191]|[-35959452, -35959452, -35959452, -35959452, -35959452, -35959452]|[-4256846, -4256846, -4256846, -4256846, -4256846, -4256846]|[2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24]|[2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43]|[2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05]|[2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04]|['V', 'V', 'V', 'V', 'V', 'V']|['', '', '', '', '', '']|['8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj', '8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj'] +17|[1, 1, 1, 1, 1, 1]|[126, 126, 126, 126, 126, 126]|[28165, 28165, 28165, 28165, 28165, 28165]|[2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721]|[-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879]|[-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876]|[-21681.223, -21681.223, -21681.223, -21681.223, -21681.223]|[95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678]|[89062729, 89062729, 89062729, 89062729, 89062729, 89062729]|[-2464406, -2464406, -2464406, -2464406, -2464406, -2464406]|[2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16]|[2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36]|[2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20]|[2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44]|['U', 'U', 'U', 'U', 'U', 'U']|['', '', '', '', '', '']|['XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb', 'XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb'] +2|[0, 0, 0, 0, 0, 0]|[117, 117, 117, 117, 117, 117]|[-4744, -4744, -4744, -4744, -4744, -4744]|[-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961]|[-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780]|[8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567]|[-30948.857, -30948.857, -30948.857, -30948.857, -30948.857]|[804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905]|[-74019648, -74019648, -74019648, -74019648, -74019648, -74019648]|[13024168, 13024168, 13024168, 13024168, 13024168, 13024168]|[2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22]|[2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12]|[2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21]|[2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56]|['g', 'g', 'g', 'g', 'g', 'g']|['', '', '', '', '', '']|['S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl', 'S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl'] diff --git a/regression-test/data/load_p0/stream_load/basic_data.csv b/regression-test/data/load_p0/stream_load/basic_data.csv new file mode 100644 index 00000000000000..daa9ebd355e8a9 --- /dev/null +++ b/regression-test/data/load_p0/stream_load/basic_data.csv @@ -0,0 +1,20 @@ +57|2023-08-19|TRUE|2|-25462|-74112029|6458082754318544493|-7910671781690629051|-15205.859375|-306870797.484914|759730669.0|-628556336.0|2023-07-10 18:39:10|2023-02-12|2023-01-27 07:26:06|y||Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ|{"name": "John", "age": 25, "city": "New York"} +49|2023-08-08|FALSE|\N|16275|-2144851675|-2303421957908954634|-46526938720058765|-13141.142578|-686632233.230200|229942298.0|-152553823.0|2022-09-01 00:16:01|2023-03-25|2022-09-07 14:59:03|s||yvuILR2iNxfe8RRml|{"student": true, "name": "Alice", "grade": 9, "subjects": ["math", "science", "history"]} +66|2023-08-15|TRUE|-91|28378|609923317|4872185586197131212|1207709464099378591|\N|-1863683325.985123|-783792012.0|-708986976.0|2022-09-24 10:39:23|2022-09-24|2022-10-16 18:36:43|Y|z|AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6|{"book": {"title": "The Great Gatsby", "author": "F. Scott Fitzgerald"}, "year": 1925} +91|2023-08-27|TRUE|90|2465|702240964|6373830997821598984|305860046137409400|15991.356445|1599972327.386147|-165530947.0|\N|2023-04-26 19:31:10|2023-07-21|\N|2||B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK|{"fruit": "apple", "color": "red", "qty": 5, "price": 2.5} +80|2023-08-18|FALSE|-18|-8971|679027874|6535956962935330265|3960889045799757165|-13219.759766|1187161924.505394|-526615878.0|-947410627.0|2023-03-11 07:40:00|2022-11-29|2023-01-14 07:24:07|\N|D|3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve|{"car": "BMW", "model": "X5", "year": 2020, "color": "black"} +85|2023-08-11|TRUE|-7|24304|-2043877415|-2024144417867729183|\N|5363.024414|-578615669.042831|-378574346.0|-810302932.0|2023-07-15 01:07:41|2023-08-13|2023-01-20 11:57:48|i||WQ9dh9ajPu0y|{"country": "France", "capital": "Paris", "population": 67081000} +31|2023-08-27|FALSE|17|-18849|1728109133|3266501886640700374|527195452623418935|-24062.328125|-1514348021.262435|-322205854.0|-278237157.0|2022-10-07 03:24:23|2022-09-25|\N|0|8|yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4|{"team": "Manchester United", "players": ["Ronaldo", "Rooney", "Giggs"], "coach": "Ole Gunnar Solskjaer"} +20|2023-08-17|FALSE|-5|18158|784479801|1485484354598941738|-6632681928222776815|9708.430664|-330432620.706069|-816424174.0|571112646.0|2022-09-15 21:40:55|2023-02-23|2023-08-13 21:31:54|O|X|2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn|{"name": "Sarah", "age": 30, "city": "London", "isMarried": false} +90|2023-08-27|TRUE|22|16456|-1476824962|-3279894870153540825|8990195191470116763|26651.906250|206860148.942546|-580959198.0|-210329147.0|2022-10-07 03:11:03|2023-03-18|2023-04-15 00:38:33|T|L|QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD|{"company": "Apple", "products": [{"name": "iPhone", "price": 1000}, {"name": "MacBook", "price": 1500}]} +8|2023-08-14|TRUE|109|-31573|-1362465190|3990845741226497177|2732763251146840270|-25698.552734|1312831962.567818|771983879.0|173937916.0|2023-03-07 14:13:19|2022-10-18|2023-07-16 05:03:13|D||PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme|{"animal": "lion", "weight": 200, "habitat": ["savannah", "grassland"]} +65|2023-08-09|FALSE|94|31514|814994517|-297697460695940343|734910652450318597|-13061.891602|62750847.041706|-9808654.0|\N|2023-08-14 22:01:27|2023-05-19|2022-11-13 13:44:28|V||aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf|{"language": "Python", "version": 3.9, "frameworks": ["Django", "Flask"]} +62|2023-08-21|FALSE|81|20302|-200761532|6365479976421007608|\N|-29916.533203|1709141750.828478|549873536.0|-119205359.0|2023-05-04 01:14:51|2022-09-17|2022-12-04 19:30:09|d|v|BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD|{"username": "user123", "password": "pass123", "email": "user123@example.com"} +50|2023-08-06|TRUE|109|-6330|1479023892|-8630800697573159428|-1645095773540208759|17880.960938|-1453844792.013949|-158871820.0|-862940384.0|2022-09-22 02:03:21|2023-05-14|2023-03-25 02:18:34|m||JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl|{"city": "Tokyo", "temperature": 20.5, "humidity": 75} +58|2023-08-22|\N|0|-18231|1832867360|6997858407575297145|2480714305422728023|-5450.488770|1475901032.138386|-893480655.0|-607891858.0|2023-02-02 05:13:24|2022-09-18|2023-04-23 10:51:15|k||LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja|{"restaurant": "Pizza Hut", "menu": ["pizza", "pasta", "salad"]} +60|2023-08-27|FALSE|-52|-2338|-757056972|1047567408607120856|6541476642780646552|6614.089355|-1204448798.517855|236657733.0|731515433.0|2022-12-29 14:47:30|2022-09-24|2023-08-01 12:41:59|O|F|RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU|{"game": "Chess", "players": 2, "time": "1 hour"} +68|2023-08-23|TRUE|-73|20117|1737338128|795638676048937749|-5551546237562433901|-30627.039062|68589475.684545|585022347.0|513722420.0|2022-12-28 20:26:51|2022-10-04|2023-07-30 00:20:06|y||keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM|{"country": "Brazil", "continent": "South America", "population": 211049527} +50|2023-08-24|TRUE|15|14403|\N|-6418906115745394180|9205303779366462513|-4331.548828|-615112179.557648|367305015.0|-551652958.0|2022-12-29 02:27:20|2023-06-01|2023-08-12 04:50:04|a||eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM|{"band": "The Beatles", "members": ["John Lennon", "Paul McCartney", "George Harrison", "Ringo Starr"]} +81|2023-08-23|FALSE|106|11492|-667795397|4480250461471356146|-5346660566234294101|9082.750000|385167225.902608|-717553011.0|649146853.0|2023-03-20 03:33:16|2022-11-24|2023-02-16 18:29:41|G|9|Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag|{"flower": "rose", "color": "red", "fragrance": true} +41|2023-08-27|TRUE|-104|22750|\N|8527773271030840740|5554497317268279215|-5296.828125|-1715646888.013040|-306075962.0|897769189.0|2022-12-02 17:56:44|2022-10-12|2023-02-19 07:02:54|V|\N|E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V|{"food": "Sushi", "price": 10, "restaurant": "Sushi King"} +21|2023-08-18|FALSE|63|-27847|-35409596|8638201997392767650|4919963231735304178|-23382.541016|-1803403621.426313|-22009767.0|661750756.0|2023-03-31 10:56:14|2023-01-20|2023-02-18 13:37:52|N|T|PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi|{"city": "Sydney", "population": 5312000, "area": 2058.7} diff --git a/regression-test/data/load_p0/stream_load/basic_data.csv.bz2 b/regression-test/data/load_p0/stream_load/basic_data.csv.bz2 new file mode 100644 index 00000000000000..16984c2ff675fd Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_data.csv.bz2 differ diff --git a/regression-test/data/load_p0/stream_load/basic_data.csv.deflate b/regression-test/data/load_p0/stream_load/basic_data.csv.deflate new file mode 100644 index 00000000000000..7f19a682c34465 Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_data.csv.deflate differ diff --git a/regression-test/data/load_p0/stream_load/basic_data.csv.gz b/regression-test/data/load_p0/stream_load/basic_data.csv.gz new file mode 100644 index 00000000000000..4e1e70481928eb Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_data.csv.gz differ diff --git a/regression-test/data/load_p0/stream_load/basic_data.csv.lz4 b/regression-test/data/load_p0/stream_load/basic_data.csv.lz4 new file mode 100644 index 00000000000000..7c1ea5a9f85955 Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_data.csv.lz4 differ diff --git a/regression-test/data/load_p0/stream_load/basic_data.csv.lzo b/regression-test/data/load_p0/stream_load/basic_data.csv.lzo new file mode 100644 index 00000000000000..4178095dcd337e Binary files /dev/null and b/regression-test/data/load_p0/stream_load/basic_data.csv.lzo differ diff --git a/regression-test/data/load_p0/stream_load/basic_data_timezone.csv b/regression-test/data/load_p0/stream_load/basic_data_timezone.csv new file mode 100644 index 00000000000000..d4fbc5967d8bbe --- /dev/null +++ b/regression-test/data/load_p0/stream_load/basic_data_timezone.csv @@ -0,0 +1,20 @@ +2023-08-19|TRUE|2|-25462|-74112029|6458082754318544493|-7910671781690629051|-15205.859375|-306870797.484914|759730669.0|-628556336.0|2023-07-10 18:39:10|2023-02-12|2023-01-27 07:26:06|y||Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ|{"name": "John", "age": 25, "city": "New York"} +2023-08-08|FALSE|\N|16275|-2144851675|-2303421957908954634|-46526938720058765|-13141.142578|-686632233.230200|229942298.0|-152553823.0|2022-09-01 00:16:01|2023-03-25|2022-09-07 14:59:03|s||yvuILR2iNxfe8RRml|{"student": true, "name": "Alice", "grade": 9, "subjects": ["math", "science", "history"]} +2023-08-15|TRUE|-91|28378|609923317|4872185586197131212|1207709464099378591|\N|-1863683325.985123|-783792012.0|-708986976.0|2022-09-24 10:39:23|2022-09-24|2022-10-16 18:36:43|Y|z|AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6|{"book": {"title": "The Great Gatsby", "author": "F. Scott Fitzgerald"}, "year": 1925} +2023-08-27|TRUE|90|2465|702240964|6373830997821598984|305860046137409400|15991.356445|1599972327.386147|-165530947.0|\N|2023-04-26 19:31:10|2023-07-21|\N|2||B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK|{"fruit": "apple", "color": "red", "qty": 5, "price": 2.5} +2023-08-18|FALSE|-18|-8971|679027874|6535956962935330265|3960889045799757165|-13219.759766|1187161924.505394|-526615878.0|-947410627.0|2023-03-11 07:40:00|2022-11-29|2023-01-14 07:24:07|\N|D|3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve|{"car": "BMW", "model": "X5", "year": 2020, "color": "black"} +2023-08-11|TRUE|-7|24304|-2043877415|-2024144417867729183|\N|5363.024414|-578615669.042831|-378574346.0|-810302932.0|2023-07-15 01:07:41|2023-08-13|2023-01-20 11:57:48|i||WQ9dh9ajPu0y|{"country": "France", "capital": "Paris", "population": 67081000} +2023-08-27|FALSE|17|-18849|1728109133|3266501886640700374|527195452623418935|-24062.328125|-1514348021.262435|-322205854.0|-278237157.0|2022-10-07 03:24:23|2022-09-25|\N|0|8|yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4|{"team": "Manchester United", "players": ["Ronaldo", "Rooney", "Giggs"], "coach": "Ole Gunnar Solskjaer"} +2023-08-17|FALSE|-5|18158|784479801|1485484354598941738|-6632681928222776815|9708.430664|-330432620.706069|-816424174.0|571112646.0|2022-09-15 21:40:55|2023-02-23|2023-08-13 21:31:54|O|X|2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn|{"name": "Sarah", "age": 30, "city": "London", "isMarried": false} +2023-08-27|TRUE|22|16456|-1476824962|-3279894870153540825|8990195191470116763|26651.906250|206860148.942546|-580959198.0|-210329147.0|2022-10-07 03:11:03|2023-03-18|2023-04-15 00:38:33|T|L|QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD|{"company": "Apple", "products": [{"name": "iPhone", "price": 1000}, {"name": "MacBook", "price": 1500}]} +2023-08-14|TRUE|109|-31573|-1362465190|3990845741226497177|2732763251146840270|-25698.552734|1312831962.567818|771983879.0|173937916.0|2023-03-07 14:13:19|2022-10-18|2023-07-16 05:03:13|D||PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme|{"animal": "lion", "weight": 200, "habitat": ["savannah", "grassland"]} +2023-08-09|FALSE|94|31514|814994517|-297697460695940343|734910652450318597|-13061.891602|62750847.041706|-9808654.0|\N|2023-08-14 22:01:27|2023-05-19|2022-11-13 13:44:28|V||aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf|{"language": "Python", "version": 3.9, "frameworks": ["Django", "Flask"]} +2023-08-21|FALSE|81|20302|-200761532|6365479976421007608|\N|-29916.533203|1709141750.828478|549873536.0|-119205359.0|2023-05-04 01:14:51|2022-09-17|2022-12-04 19:30:09|d|v|BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD|{"username": "user123", "password": "pass123", "email": "user123@example.com"} +2023-08-06|TRUE|109|-6330|1479023892|-8630800697573159428|-1645095773540208759|17880.960938|-1453844792.013949|-158871820.0|-862940384.0|2022-09-22 02:03:21|2023-05-14|2023-03-25 02:18:34|m||JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl|{"city": "Tokyo", "temperature": 20.5, "humidity": 75} +2023-08-22|\N|0|-18231|1832867360|6997858407575297145|2480714305422728023|-5450.488770|1475901032.138386|-893480655.0|-607891858.0|2023-02-02 05:13:24|2022-09-18|2023-04-23 10:51:15|k||LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja|{"restaurant": "Pizza Hut", "menu": ["pizza", "pasta", "salad"]} +2023-08-27|FALSE|-52|-2338|-757056972|1047567408607120856|6541476642780646552|6614.089355|-1204448798.517855|236657733.0|731515433.0|2022-12-29 14:47:30|2022-09-24|2023-08-01 12:41:59|O|F|RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU|{"game": "Chess", "players": 2, "time": "1 hour"} +2023-08-23|TRUE|-73|20117|1737338128|795638676048937749|-5551546237562433901|-30627.039062|68589475.684545|585022347.0|513722420.0|2022-12-28 20:26:51|2022-10-04|2023-07-30 00:20:06|y||keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM|{"country": "Brazil", "continent": "South America", "population": 211049527} +2023-08-24|TRUE|15|14403|\N|-6418906115745394180|9205303779366462513|-4331.548828|-615112179.557648|367305015.0|-551652958.0|2022-12-29 02:27:20|2023-06-01|2023-08-12 04:50:04|a||eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM|{"band": "The Beatles", "members": ["John Lennon", "Paul McCartney", "George Harrison", "Ringo Starr"]} +2023-08-23|FALSE|106|11492|-667795397|4480250461471356146|-5346660566234294101|9082.750000|385167225.902608|-717553011.0|649146853.0|2023-03-20 03:33:16|2022-11-24|2023-02-16 18:29:41|G|9|Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag|{"flower": "rose", "color": "red", "fragrance": true} +2023-08-27|TRUE|-104|22750|\N|8527773271030840740|5554497317268279215|-5296.828125|-1715646888.013040|-306075962.0|897769189.0|2022-12-02 17:56:44|2022-10-12|2023-02-19 07:02:54|V|\N|E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V|{"food": "Sushi", "price": 10, "restaurant": "Sushi King"} +2023-08-18|FALSE|63|-27847|-35409596|8638201997392767650|4919963231735304178|-23382.541016|-1803403621.426313|-22009767.0|661750756.0|2023-03-31 10:56:14|2023-01-20|2023-02-18 13:37:52|N|T|PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi|{"city": "Sydney", "population": 5312000, "area": 2058.7} diff --git a/regression-test/data/load_p0/stream_load/basic_data_with_errors.csv b/regression-test/data/load_p0/stream_load/basic_data_with_errors.csv new file mode 100644 index 00000000000000..8452e9513fd2ff --- /dev/null +++ b/regression-test/data/load_p0/stream_load/basic_data_with_errors.csv @@ -0,0 +1,20 @@ +|2023-08-19|TRUE|2|-25462|-74112029|6458082754318544493|-7910671781690629051|-15205.859375|-306870797.484914|759730669.0|-628556336.0|2023-07-10 18:39:10|2023-02-12|2023-01-27 07:26:06|y||Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ|{"name": "John", "age": 25, "city": "New York"} +49|2023-08-08|FALSE|16275|-2144851675|-2303421957908954634|-46526938720058765|-13141.142578|-686632233.230200|229942298.0|-152553823.0|2022-09-01 00:16:01|2023-03-25|2022-09-07 14:59:03|s||yvuILR2iNxfe8RRml|{"student": true, "name": "Alice", "grade": 9, "subjects": ["math", "science", "history"]} +66|2023-08-15|TRUE|redundant|-91|28378|609923317|4872185586197131212|1207709464099378591|\N|-1863683325.985123|-783792012.0|-708986976.0|2022-09-24 10:39:23|2022-09-24|2022-10-16 18:36:43|Y|z|AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6|{"book": {"title": "The Great Gatsby", "author": "F. Scott Fitzgerald"}, "year": 1925} +91|2023-08-27|TRUE|90000000000000|2465|702240964|6373830997821598984|305860046137409400|15991.356445|1599972327.386147|-165530947.0|\N|2023-04-26 19:31:10|2023-07-21|\N|2||B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK|{"fruit": "apple", "color": "red", "qty": 5, "price": 2.5} +80|2023-08-18|FALSE|-18|-8971|679027874|6535956962935330265|3960889045799757165|-13219.759766|1187161924.505394|-526615878.0|-947410627.0|2023-03-11 07:40:00|2022-11-29|2023-01-14 07:24:07|\N|D|3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve|{"car": "BMW", "model": "X5", "year": 2020, "color": "black"} +85|2023-08-11 10:10:10|TRUE|-7|24304|-2043877415|-2024144417867729183|\N|5363.024414|-578615669.042831|-378574346.0|-810302932.0|2023-07-15 01:07:41|2023-08-13|2023-01-20 11:57:48|i||WQ9dh9ajPu0y|{"country": "France", "capital": "Paris", "population": 67081000} +31|2023-08-27|FALSE|17|-18849|1728109133|3266501886640700374|527195452623418935|-24062.328125|-1514348021.262435|-322205854.0|-278237157.0|2022-10-07 03:24:23|2022-09-25|\N|0|8|yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4|{"team": "Manchester United", "players": ["Ronaldo", "Rooney", "Giggs"], "coach": "Ole Gunnar Solskjaer"} +20|2023-08-17|FALSE|-5|18158|784479801|1485484354598941738|-6632681928222776815|9708.430664|-330432620.706069|-816424174.0|571112646.00000|2022-09-15 21:40:55|2023-02-23|2023-08-13 21:31:54|O|X|2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn|{"name": "Sarah", "age": 30, "city": "London", "isMarried": false} +90|2023-08-27|TRUE|22|16456|-1476824962|-3279894870153540825|8990195191470116763e|26651.906250|206860148.942546|-580959198.0|-210329147.0|2022-10-07 03:11:03|2023-03-18|2023-04-15 00:38:33|T|L|QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD|{"company": "Apple", "products": [{"name": "iPhone", "price": 1000}, {"name": "MacBook", "price": 1500}]} +8|2023-08-14|TRUE|109|-31573|-1362465190|3990845741226497177|2732763251146840270|-25698.552734|1312831962.567818|771983879.0|173937916.0|2023-03-07 14:13:19|2022-10-18|2023-07-16 05:03:13|D||PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme|{"animal": "lion", "weight": 200, "habitat": ["savannah", "grassland"]} +65|2023-08-09|FALSE|94|31514|814994517|-297697460695940343|734910652450318597|-13061.891602|62750847.041706|-9808654.0|\N|2023-08-14 22:01:27|2023-05-19|2022-11-13 13:44:28|V||aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf|{"language": "Python" "version": 3.9, "frameworks": ["Django", "Flask"]} +62|2023-08-21|FALSE|81|20302|-200761532|6365479976421007608|\N|-29916.533203|1709141750.828478|549873536.0|-119205359.0|2023-05-04 01:14:51|2022-09-17|2022-12-04 19:30:09|d|v|BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD|{"username": "user123", "password": "pass123", "email": "user123@example.com" +50|2023-08-06|TRUE|109|-6330|1479023892|-8630800697573159428|-1645095773540208759|17880.960938|-1453844792.013949|-158871820.0|-862940384.0|2022-09-22 02:03:21|2023-05-14|2023-03-25 02:18:34|m||JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl|{"city": "Tokyo", "temperature": 20.5, "humidity": 75} +58|2023-08-22|\N|0|-18231|1832867360|6997858407575297145|2480714305422728023|-5450.488770|1475901032.138386|-893480655.0|-607891858.0|2023-02-02 05:13:24|2022-09-18|2023-04-23 10:51:15|k||LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja|{"restaurant": "Pizza Hut", ["pizza", "pasta", "salad"]} +60|2023-08-27|FALSE|-52|-2338|-757056972|1047567408607120856|6541476642780646552|6614.089355|-1204448798.517855|236657733.0|731515433.0|2022-12-29 14:47:30|2022-09-24|2023-08-01 12:41:59|O|F|RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU|{"game": "Chess", "players": 2, "time": "1 hour"} +68|2023-08-23|TRUE|-73|20117|1737338128|795638676048937749|-5551546237562433901|-30627.039062|68589475.684545|585022347.0|513722420.0|2022-12-28 20:26:51|2022-10-04|2023-07-30 00:20:06|y||keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM|{"country": "Brazil", "continent": "South America", "population": 211049527} +50|2023-08-24|TRUE|15|14403|\N|-6418906115745394180|9205303779366462513|-4331.548828|-615112179.557648|367305015.0|-551652958.0|2022-12-29 02:27:20|2023-06-01|2023-08-12 04:50:04|a||eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM|{"band": "The Beatles", "members": ["John Lennon", "Paul McCartney", "George Harrison", "Ringo Starr"]} +81|2023-08-23|FALSE|106|11492|-667795397|4480250461471356146|-5346660566234294101|9082.750000|385167225.902608|-717553011.0|649146853.0|2023-03-20 03:33:16|2022-11-24|2023-02-16 18:29:41|G|9|Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag|{"flower": "rose", "color": "red", "fragrance": true} +41|2023-08-27|TRUE|-104|22750|\N|8527773271030840740|5554497317268279215|-5296.828125|-1715646888.013040|-306075962.0|897769189.0|2022-12-02 17:56:44|2022-10-12|2023-02-19 07:02:54|V|\N|E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V|{"food": "Sushi", "price": 10, "restaurant": "Sushi King"} +21|2023-08-18|FALSE|63|-27847|-35409596|8638201997392767650|4919963231735304178|-23382.541016|-1803403621.426313|-22009767.0|661750756.0|2023-03-31 10:56:14|2023-01-20|2023-02-18 13:37:52|N|T|PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi|{"city": "Sydney", "population": 5312000, "area": 2058.7} diff --git a/regression-test/data/load_p0/stream_load/test_compress.csv.bz2 b/regression-test/data/load_p0/stream_load/test_compress.csv.bz2 new file mode 100644 index 00000000000000..b2fd1fcfbe6af1 Binary files /dev/null and b/regression-test/data/load_p0/stream_load/test_compress.csv.bz2 differ diff --git a/regression-test/data/load_p0/stream_load/test_compress.csv.gz b/regression-test/data/load_p0/stream_load/test_compress.csv.gz new file mode 100644 index 00000000000000..a330a2b07194dd Binary files /dev/null and b/regression-test/data/load_p0/stream_load/test_compress.csv.gz differ diff --git a/regression-test/data/load_p0/stream_load/test_compress.csv.lz4 b/regression-test/data/load_p0/stream_load/test_compress.csv.lz4 new file mode 100644 index 00000000000000..76955306d8ae66 Binary files /dev/null and b/regression-test/data/load_p0/stream_load/test_compress.csv.lz4 differ diff --git a/regression-test/data/load_p0/stream_load/test_group_commit_stream_load.out b/regression-test/data/load_p0/stream_load/test_group_commit_stream_load.out new file mode 100644 index 00000000000000..246be06453bd16 --- /dev/null +++ b/regression-test/data/load_p0/stream_load/test_group_commit_stream_load.out @@ -0,0 +1,27 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 a 10 +1 a 10 +1 a 10 +2 b 20 +2 b 20 +2 b 20 +3 c 30 +3 c 30 +3 c 30 +4 d \N +4 d \N +4 d \N +5 e -1 +5 e 50 +6 f -1 +6 f -1 +6 f 60 +7 e 70 +8 f 80 +10 a 10 +11 a 11 + +-- !sql -- +2402288 + diff --git a/regression-test/data/load_p0/stream_load/test_stream_load1.csv b/regression-test/data/load_p0/stream_load/test_stream_load1.csv new file mode 100644 index 00000000000000..b86bcfa176fe18 --- /dev/null +++ b/regression-test/data/load_p0/stream_load/test_stream_load1.csv @@ -0,0 +1,2 @@ +5,e +6,f diff --git a/regression-test/data/load_p0/stream_load/test_stream_load2.csv b/regression-test/data/load_p0/stream_load/test_stream_load2.csv new file mode 100644 index 00000000000000..93157e3d6ddd53 --- /dev/null +++ b/regression-test/data/load_p0/stream_load/test_stream_load2.csv @@ -0,0 +1,2 @@ +70|7|e +80|8|f diff --git a/regression-test/data/load_p0/stream_load/test_stream_load3.csv b/regression-test/data/load_p0/stream_load/test_stream_load3.csv new file mode 100644 index 00000000000000..f257be1566f9e9 --- /dev/null +++ b/regression-test/data/load_p0/stream_load/test_stream_load3.csv @@ -0,0 +1,6 @@ +10,a,10 +11,a,11 +1a,a,11 +12,a +20,b,21 +101,a,101 \ No newline at end of file diff --git a/regression-test/data/load_p0/stream_load/test_stream_load_properties.out b/regression-test/data/load_p0/stream_load/test_stream_load_properties.out new file mode 100644 index 00000000000000..da5f574bdbb29e --- /dev/null +++ b/regression-test/data/load_p0/stream_load/test_stream_load_properties.out @@ -0,0 +1,640 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql_timezone_shanghai -- +1196389819 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_timezone_shanghai -- +1196389819 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_timezone_shanghai -- +1196389819 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196389819 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_timezone_shanghai -- +1196389819 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-18 false 45 28718 643618278 -3272585113381453701 8880852277535061343 -36602.3 -6.16241696920919E8 -122009766.9 0.0 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} \N \N \N true 2 4 6 8 10 12.0 14.0 1777777776 1999999998 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-23 false 33 31609 1069542731 5275889137520293895 -10898206803796728002 -21544.29 4.5375670158715296E8 0.0 199999999.8 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} \N \N \N true 2 4 6 8 10 12.0 14.0 1777777776 1999999998 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196389819 2023-08-27 true -27 20484 196468163 -2510965379762831487 21919224650279870865 19898.195 -2.627611231464637E9 129496729.9 0.0 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} \N \N \N true 5 10 15 20 25 30.0 35.0 149477144 705032699 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N + +-- !sql_timezone_shanghai -- +1196389819 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [1, 1, 1, 1, 1, 1] [126, 126, 126, 126, 126, 126] [28165, 28165, 28165, 28165, 28165, 28165] [2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721] [-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879] [-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876] [-21681.223, -21681.223, -21681.223, -21681.223, -21681.223] [95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678] [89062729, 89062729, 89062729, 89062729, 89062729, 89062729] [-2464406, -2464406, -2464406, -2464406, -2464406, -2464406] [2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16] [2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36] [2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20] [2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [-43, -43, -43, -43, -43, -43] [13560, 13560, 13560, 13560, 13560, 13560] [-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513] [7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769] [-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413] [12225.427, 12225.427, 12225.427, 12225.427, 12225.427] [-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191] [-35959452, -35959452, -35959452, -35959452, -35959452, -35959452] [-4256846, -4256846, -4256846, -4256846, -4256846, -4256846] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43] [2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05] [2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04] ["V", "V", "V", "V", "V", "V"] ["a", "a", "a", "a", "a", "a"] ["8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [1, 1, 1, 1, 1, 1] [-7, -7, -7, -7, -7, -7] [18655, 18655, 18655, 18655, 18655, 18655] [6240356, 6240356, 6240356, 6240356, 6240356, 6240356] [4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393] [323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978] [2972.2478, 2972.2478, 2972.2478, 2972.2478, 2972.2478] [-1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008] [-84949097, -84949097, -84949097, -84949097, -84949097, -84949097] [29815991, 29815991, 29815991, 29815991, 29815991, 29815991] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11] [2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10] [2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07] ["9", "9", "9", "9", "9", "9"] ["n", "n", "n", "n", "n", "n"] ["s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [-25, -25, -25, -25, -25, -25] [28704, 28704, 28704, 28704, 28704, 28704] [-437867812, -437867812, -437867812, -437867812, -437867812, -437867812] [5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079] [462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597] [13629.614, 13629.614, 13629.614, 13629.614, 13629.614] [213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962] [86704879, 86704879, 86704879, 86704879, 86704879, 86704879] [-2662959, -2662959, -2662959, -2662959, -2662959, -2662959] [2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04] [2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08] [2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13] [2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07] ["4", "4", "4", "4", "4", "4"] ["a", "a", "a", "a", "a", "a"] ["2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [-6985, -6985, -6985, -6985, -6985, -6985] [826683531, 826683531, 826683531, 826683531, 826683531, 826683531] [-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361] [4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446] [-6490.2471, -6490.2471, -6490.2471, -6490.2471, -6490.2471] [1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261] [-25002204, -25002204, -25002204, -25002204, -25002204, -25002204] [-97805693, -97805693, -97805693, -97805693, -97805693, -97805693] [2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02] [2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22] [2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19] [2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09] ["p", "p", "p", "p", "p", "p"] ["a", "a", "a", "a", "a", "a"] ["WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [18307, 18307, 18307, 18307, 18307, 18307] [76399879, 76399879, 76399879, 76399879, 76399879, 76399879] [8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699] [2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999] [-16678.924, -16678.924, -16678.924, -16678.924, -16678.924] [-2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261] [12639967, 12639967, 12639967, 12639967, 12639967, 12639967] [24677022, 24677022, 24677022, 24677022, 24677022, 24677022] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55] [2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31] [2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44] ["d", "d", "d", "d", "d", "d"] ["a", "a", "a", "a", "a", "a"] ["dGkS", "dGkS", "dGkS", "dGkS", "dGkS", "dGkS"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [1, 1, 1, 1, 1, 1] [23, 23, 23, 23, 23, 23] [14833, 14833, 14833, 14833, 14833, 14833] [2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015] [7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930] [-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347] [-29817.77, -29817.77, -29817.77, -29817.77, -29817.77] [-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042] [44414137, 44414137, 44414137, 44414137, 44414137, 44414137] [75659236, 75659236, 75659236, 75659236, 75659236, 75659236] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19] [2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27] [2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51] ["a", "a", "a", "a", "a", "a"] ["a", "a", "a", "a", "a", "a"] ["gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [-126, -126, -126, -126, -126, -126] [-3813, -3813, -3813, -3813, -3813, -3813] [1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540] [-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217] [5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397] [-17121.195, -17121.195, -17121.195, -17121.195, -17121.195] [-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562] [9626406, 9626406, 9626406, 9626406, 9626406, 9626406] [-8723076, -8723076, -8723076, -8723076, -8723076, -8723076] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20] [2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06] [2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10] ["B", "B", "B", "B", "B", "B"] ["a", "a", "a", "a", "a", "a"] ["8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [1, 1, 1, 1, 1, 1] [121, 121, 121, 121, 121, 121] [31806, 31806, 31806, 31806, 31806, 31806] [-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562] [-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717] [4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431] [10719.892, 10719.892, 10719.892, 10719.892, 10719.892] [1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421] [78876206, 78876206, 78876206, 78876206, 78876206, 78876206] [-77837482, -77837482, -77837482, -77837482, -77837482, -77837482] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07] [2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15] [2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [-67, -67, -67, -67, -67, -67] [-30372, -30372, -30372, -30372, -30372, -30372] [181502941, 181502941, 181502941, 181502941, 181502941, 181502941] [-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800] [6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842] [21235.783, 21235.783, 21235.783, 21235.783, 21235.783] [-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891] [58088067, 58088067, 58088067, 58088067, 58088067, 58088067] [-66079723, -66079723, -66079723, -66079723, -66079723, -66079723] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12] [2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19] [2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [63, 63, 63, 63, 63, 63] [11399, 11399, 11399, 11399, 11399, 11399] [-583523026, -583523026, -583523026, -583523026, -583523026, -583523026] [8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293] [-8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906] [23243.16, 23243.16, 23243.16, 23243.16, 23243.16] [716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115] [-40335903, -40335903, -40335903, -40335903, -40335903, -40335903] [74087997, 74087997, 74087997, 74087997, 74087997, 74087997] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27] [2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28] [2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47] ["z", "z", "z", "z", "z", "z"] ["1", "1", "1", "1", "1", "1"] ["xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, 61] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [0, 0, 0, 0, 0, 0] [-10, -10, -10, -10, -10, -10] [9493, 9493, 9493, 9493, 9493, 9493] [-547874696, -547874696, -547874696, -547874696, -547874696, -547874696] [-115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756] [4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085] [13718.372, 13718.372, 13718.372, 13718.372, 13718.372] [-978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697] [-88516589, -88516589, -88516589, -88516589, -88516589, -88516589] [-62683124, -62683124, -62683124, -62683124, -62683124, -62683124] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43] [2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25] [2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14] ["J", "J", "J", "J", "J", "J"] ["a", "a", "a", "a", "a", "a"] ["yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196389819 [1, 1, 1, 1, 1, 1] [65, 65, 65, 65, 65, 65] [18805, 18805, 18805, 18805, 18805, 18805] [229725878, 229725878, 229725878, 229725878, 229725878, 229725878] [2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325] [5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371] [12354.624, 12354.624, 12354.624, 12354.624, 12354.624] [1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771] [20409908, 20409908, 20409908, 20409908, 20409908, 20409908] [-69625379, -69625379, -69625379, -69625379, -69625379, -69625379] [2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09] [2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54] [2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19] [2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_timezone_shanghai -- +1196389819 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_timezone_shanghai -- +1196389819 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_timezone_Abidjan -- +1196418619 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_timezone_Abidjan -- +1196418619 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_timezone_Abidjan -- +1196418619 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +1196418619 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_timezone_Abidjan -- +1196418619 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-18 false 45 28718 643618278 -3272585113381453701 8880852277535061343 -36602.3 -6.16241696920919E8 -122009766.9 0.0 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} \N \N \N true 2 4 6 8 10 12.0 14.0 1777777776 1999999998 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-23 false 33 31609 1069542731 5275889137520293895 -10898206803796728002 -21544.29 4.5375670158715296E8 0.0 199999999.8 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} \N \N \N true 2 4 6 8 10 12.0 14.0 1777777776 1999999998 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +1196418619 2023-08-27 true -27 20484 196468163 -2510965379762831487 21919224650279870865 19898.195 -2.627611231464637E9 129496729.9 0.0 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} \N \N \N true 5 10 15 20 25 30.0 35.0 149477144 705032699 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N + +-- !sql_timezone_Abidjan -- +1196418619 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [1, 1, 1, 1, 1, 1] [126, 126, 126, 126, 126, 126] [28165, 28165, 28165, 28165, 28165, 28165] [2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721] [-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879] [-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876] [-21681.223, -21681.223, -21681.223, -21681.223, -21681.223] [95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678] [89062729, 89062729, 89062729, 89062729, 89062729, 89062729] [-2464406, -2464406, -2464406, -2464406, -2464406, -2464406] [2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16] [2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36] [2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20] [2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [-43, -43, -43, -43, -43, -43] [13560, 13560, 13560, 13560, 13560, 13560] [-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513] [7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769] [-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413] [12225.427, 12225.427, 12225.427, 12225.427, 12225.427] [-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191] [-35959452, -35959452, -35959452, -35959452, -35959452, -35959452] [-4256846, -4256846, -4256846, -4256846, -4256846, -4256846] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43] [2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05] [2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04] ["V", "V", "V", "V", "V", "V"] ["a", "a", "a", "a", "a", "a"] ["8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [1, 1, 1, 1, 1, 1] [-7, -7, -7, -7, -7, -7] [18655, 18655, 18655, 18655, 18655, 18655] [6240356, 6240356, 6240356, 6240356, 6240356, 6240356] [4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393] [323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978] [2972.2478, 2972.2478, 2972.2478, 2972.2478, 2972.2478] [-1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008] [-84949097, -84949097, -84949097, -84949097, -84949097, -84949097] [29815991, 29815991, 29815991, 29815991, 29815991, 29815991] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11] [2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10] [2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07] ["9", "9", "9", "9", "9", "9"] ["n", "n", "n", "n", "n", "n"] ["s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [-25, -25, -25, -25, -25, -25] [28704, 28704, 28704, 28704, 28704, 28704] [-437867812, -437867812, -437867812, -437867812, -437867812, -437867812] [5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079] [462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597] [13629.614, 13629.614, 13629.614, 13629.614, 13629.614] [213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962] [86704879, 86704879, 86704879, 86704879, 86704879, 86704879] [-2662959, -2662959, -2662959, -2662959, -2662959, -2662959] [2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04] [2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08] [2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13] [2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07] ["4", "4", "4", "4", "4", "4"] ["a", "a", "a", "a", "a", "a"] ["2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [-6985, -6985, -6985, -6985, -6985, -6985] [826683531, 826683531, 826683531, 826683531, 826683531, 826683531] [-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361] [4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446] [-6490.2471, -6490.2471, -6490.2471, -6490.2471, -6490.2471] [1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261] [-25002204, -25002204, -25002204, -25002204, -25002204, -25002204] [-97805693, -97805693, -97805693, -97805693, -97805693, -97805693] [2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02] [2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22] [2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19] [2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09] ["p", "p", "p", "p", "p", "p"] ["a", "a", "a", "a", "a", "a"] ["WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [18307, 18307, 18307, 18307, 18307, 18307] [76399879, 76399879, 76399879, 76399879, 76399879, 76399879] [8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699] [2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999] [-16678.924, -16678.924, -16678.924, -16678.924, -16678.924] [-2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261] [12639967, 12639967, 12639967, 12639967, 12639967, 12639967] [24677022, 24677022, 24677022, 24677022, 24677022, 24677022] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55] [2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31] [2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44] ["d", "d", "d", "d", "d", "d"] ["a", "a", "a", "a", "a", "a"] ["dGkS", "dGkS", "dGkS", "dGkS", "dGkS", "dGkS"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [1, 1, 1, 1, 1, 1] [23, 23, 23, 23, 23, 23] [14833, 14833, 14833, 14833, 14833, 14833] [2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015] [7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930] [-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347] [-29817.77, -29817.77, -29817.77, -29817.77, -29817.77] [-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042] [44414137, 44414137, 44414137, 44414137, 44414137, 44414137] [75659236, 75659236, 75659236, 75659236, 75659236, 75659236] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19] [2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27] [2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51] ["a", "a", "a", "a", "a", "a"] ["a", "a", "a", "a", "a", "a"] ["gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [-126, -126, -126, -126, -126, -126] [-3813, -3813, -3813, -3813, -3813, -3813] [1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540] [-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217] [5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397] [-17121.195, -17121.195, -17121.195, -17121.195, -17121.195] [-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562] [9626406, 9626406, 9626406, 9626406, 9626406, 9626406] [-8723076, -8723076, -8723076, -8723076, -8723076, -8723076] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20] [2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06] [2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10] ["B", "B", "B", "B", "B", "B"] ["a", "a", "a", "a", "a", "a"] ["8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [1, 1, 1, 1, 1, 1] [121, 121, 121, 121, 121, 121] [31806, 31806, 31806, 31806, 31806, 31806] [-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562] [-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717] [4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431] [10719.892, 10719.892, 10719.892, 10719.892, 10719.892] [1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421] [78876206, 78876206, 78876206, 78876206, 78876206, 78876206] [-77837482, -77837482, -77837482, -77837482, -77837482, -77837482] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07] [2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15] [2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [-67, -67, -67, -67, -67, -67] [-30372, -30372, -30372, -30372, -30372, -30372] [181502941, 181502941, 181502941, 181502941, 181502941, 181502941] [-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800] [6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842] [21235.783, 21235.783, 21235.783, 21235.783, 21235.783] [-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891] [58088067, 58088067, 58088067, 58088067, 58088067, 58088067] [-66079723, -66079723, -66079723, -66079723, -66079723, -66079723] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12] [2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19] [2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [63, 63, 63, 63, 63, 63] [11399, 11399, 11399, 11399, 11399, 11399] [-583523026, -583523026, -583523026, -583523026, -583523026, -583523026] [8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293] [-8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906] [23243.16, 23243.16, 23243.16, 23243.16, 23243.16] [716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115] [-40335903, -40335903, -40335903, -40335903, -40335903, -40335903] [74087997, 74087997, 74087997, 74087997, 74087997, 74087997] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27] [2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28] [2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47] ["z", "z", "z", "z", "z", "z"] ["1", "1", "1", "1", "1", "1"] ["xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, 61] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [0, 0, 0, 0, 0, 0] [-10, -10, -10, -10, -10, -10] [9493, 9493, 9493, 9493, 9493, 9493] [-547874696, -547874696, -547874696, -547874696, -547874696, -547874696] [-115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756] [4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085] [13718.372, 13718.372, 13718.372, 13718.372, 13718.372] [-978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697] [-88516589, -88516589, -88516589, -88516589, -88516589, -88516589] [-62683124, -62683124, -62683124, -62683124, -62683124, -62683124] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43] [2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25] [2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14] ["J", "J", "J", "J", "J", "J"] ["a", "a", "a", "a", "a", "a"] ["yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1196418619 [1, 1, 1, 1, 1, 1] [65, 65, 65, 65, 65, 65] [18805, 18805, 18805, 18805, 18805, 18805] [229725878, 229725878, 229725878, 229725878, 229725878, 229725878] [2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325] [5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371] [12354.624, 12354.624, 12354.624, 12354.624, 12354.624] [1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771] [20409908, 20409908, 20409908, 20409908, 20409908, 20409908] [-69625379, -69625379, -69625379, -69625379, -69625379, -69625379] [2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09] [2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54] [2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19] [2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_timezone_Abidjan -- +1196418619 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_timezone_Abidjan -- +1196418619 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_strict_mode -- + +-- !sql_strict_mode -- + +-- !sql_strict_mode -- + +-- !sql_strict_mode -- + +-- !sql_strict_mode -- + +-- !sql_strict_mode -- + +-- !sql_strict_mode -- + +-- !sql_max_filter_ratio -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_max_filter_ratio -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_max_filter_ratio -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_max_filter_ratio -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N + +-- !sql_max_filter_ratio -- +3 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [63, 63, 63, 63, 63, 63] [11399, 11399, 11399, 11399, 11399, 11399] [-583523026, -583523026, -583523026, -583523026, -583523026, -583523026] [8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293] [-8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906] [23243.16, 23243.16, 23243.16, 23243.16, 23243.16] [716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115] [-40335903, -40335903, -40335903, -40335903, -40335903, -40335903] [74087997, 74087997, 74087997, 74087997, 74087997, 74087997] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-08-06 00:00:00, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27] [2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28] [2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47] ["z", "z", "z", "z", "z", "z"] ["1", "1", "1", "1", "1", "1"] ["xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +8 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [1, 1, 1, 1, 1, 1] [-7, -7, -7, -7, -7, -7] [18655, 18655, 18655, 18655, 18655, 18655] [6240356, 6240356, 6240356, 6240356, 6240356, 6240356] [4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393] [323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978] [2972.2478, 2972.2478, 2972.2478, 2972.2478, 2972.2478] [-1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008] [-84949097, -84949097, -84949097, -84949097, -84949097, -84949097] [29815991, 29815991, 29815991, 29815991, 29815991, 29815991] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11] [2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10] [2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07] ["9", "9", "9", "9", "9", "9"] ["n", "n", "n", "n", "n", "n"] ["s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +44 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, NULL] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +48 [0, 0, 0, 0, 0, NULL] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +49 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +76 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_max_filter_ratio -- +3 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [63, 63, 63, 63, 63, 63] [11399, 11399, 11399, 11399, 11399, 11399] [-583523026, -583523026, -583523026, -583523026, -583523026, -583523026] [8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293] [-8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906] [23243.16, 23243.16, 23243.16, 23243.16, 23243.16] [716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115] [-40335903, -40335903, -40335903, -40335903, -40335903, -40335903] [74087997, 74087997, 74087997, 74087997, 74087997, 74087997] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-08-06 00:00:00, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27] [2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28] [2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47] ["z", "z", "z", "z", "z", "z"] ["1", "1", "1", "1", "1", "1"] ["xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +8 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [1, 1, 1, 1, 1, 1] [-7, -7, -7, -7, -7, -7] [18655, 18655, 18655, 18655, 18655, 18655] [6240356, 6240356, 6240356, 6240356, 6240356, 6240356] [4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393] [323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978] [2972.2478, 2972.2478, 2972.2478, 2972.2478, 2972.2478] [-1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008] [-84949097, -84949097, -84949097, -84949097, -84949097, -84949097] [29815991, 29815991, 29815991, 29815991, 29815991, 29815991] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11] [2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10] [2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07] ["9", "9", "9", "9", "9", "9"] ["n", "n", "n", "n", "n", "n"] ["s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +44 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, NULL] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +48 [0, 0, 0, 0, 0, NULL] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +49 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +76 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_max_filter_ratio -- +2 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["''", "''", "''", "''", "''", "''"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +3 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [-67, -67, -67, -67, -67, -67] [-30372, -30372, -30372, -30372, -30372, -30372] [181502941, 181502941, 181502941, 181502941, 181502941, 181502941] [-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800] [6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842] [21235.783, 21235.783, 21235.783, 21235.783, 21235.783] [-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891] [58088067, 58088067, 58088067, 58088067, 5808807, 58088067] [-66079723, -66079723, -66079723, -66079723, -66079723, -66079723] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12] [2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19] [2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12] ["G", "G", "G", "G", "G", "G"] ["''", "''", "''", "''", "''", "''"] ["tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +8 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [0, 0, 0, 0, 0, 0] [-43, -43, -43, -43, -43, -43] [13560, 13560, 13560, 13560, 13560, 13560] [-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513] [7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769] [-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413] [12225.427, 12225.427, 12225.427, 12225.427, 12225.427] [-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191] [-35959452, -35959452, -35959452, -35959452, -35959452, -35959452] [-4256846, -4256846, -4256846, -4256846, -4256846, -4256846] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43] [2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05] [2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04] ["V", "V", "V", "V", "V", "V"] ["''", "''", "''", "''", "''", "''"] ["8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +17 [1, 1, 1, 1, 1, 1] [126, 126, 126, 126, 126, 126] [28165, 28165, 28165, 28165, 28165, 28165] [2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721] [-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879] [-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876] [-21681.223, -21681.223, -21681.223, -21681.223, -21681.223] [95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678] [89062729, 89062729, 89062729, 89062729, 89062729, 89062729] [-2464406, -2464406, -2464406, -2464406, -2464406, -2464406] [2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16] [2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36] [2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20] [2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44] ["U", "U", "U", "U", "U", "U"] ["''", "''", "''", "''", "''", "''"] ["XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +41 [0, 0, 0, 0, 0, 0] [-25, -25, -25, -25, -25, -25] [28704, 28704, 28704, 28704, 28704, 28704] [-437867812, -437867812, -437867812, -437867812, -437867812, -437867812] [5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079] [462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597] [13629.614, 13629.614, 13629.614, 13629.614, 13629.614] [213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962] [86704879, 86704879, 86704879, 86704879, 86704879, 86704879] [-2662959, -2662959, -2662959, -2662959, -2662959, -2662959] [2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04] [2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08] [2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13] [2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07] ["4", "4", "4", "4", "4", "4"] ["''", "''", "''", "''", "''", "''"] ["2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +44 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, NULL] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +48 [0, 0, 0, 0, 0, NULL] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +49 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +52 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [-6985, -6985, -6985, -6985, -6985, -6985] [826683531, 826683531, 826683531, 826683531, 826683531, 826683531] [-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361] [4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446] [-6490.2471, -6490.2471, -6490.2471, -6490.2471, -6490.2471] [1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261] [-25002204, -25002204, -25002204, -25002204, -25002204, -25002204] [-97805693, -97805693, -97805693, -97805693, -97805693, -97805693] [2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02] [2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22] [2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19] [2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09] ["p", "p", "p", "p", "p", "p"] ["''", "''", "''", "''", "''", "''"] ["WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +61 [1, 1, 1, 1, 1, 1] [121, 121, 121, 121, 121, 121] [31806, 31806, 31806, 31806, 31806, 31806] [-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562] [-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717] [4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431] [10719.892, 10719.892, 10719.892, 10719.892, 10719.892] [1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421] [78876206, 78876206, 78876206, 78876206, 78876206, 78876206] [-77837482, -77837482, -77, -77837482, -77837482, -77837482] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07] [2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15] [2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16] ["U", "U", "U", "U", "U", "U"] ["''", "''", "''", "''", "''", "''"] ["y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +76 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_squence -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +49 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +57 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_merge_type -- + +-- !sql_2pc -- + +-- !sql_2pc_abort -- + +-- !sql_2pc_commit -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +49 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +57 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_2pc -- + +-- !sql_2pc_abort -- + +-- !sql_2pc_commit -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +49 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +57 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_2pc -- + +-- !sql_2pc_abort -- + +-- !sql_2pc_commit -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +49 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +57 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_2pc -- + +-- !sql_2pc_abort -- + +-- !sql_2pc_commit -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +49 2023-08-08 false \N 16275 -2144851675 -2303421957908954634 -46526938720058765 -13141.143 -6.866322332302E8 99999999.9 -99999999.9 2022-09-01T00:16:01 2023-03-25 2022-09-07T14:59:03 s yvuILR2iNxfe8RRml {"student":true,"name":"Alice","grade":9,"subjects":["math","science","history"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +57 2023-08-19 true 2 -25462 -74112029 6458082754318544493 -7910671781690629051 -15205.859 -3.06870797484914E8 99999999.9 -99999999.9 2023-07-10T18:39:10 2023-02-12 2023-01-27T07:26:06 y Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ {"name":"John","age":25,"city":"New York"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N + +-- !sql_2pc -- + +-- !sql_2pc_abort -- + +-- !sql_2pc_commit -- +2 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +3 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [-67, -67, -67, -67, -67, -67] [-30372, -30372, -30372, -30372, -30372, -30372] [181502941, 181502941, 181502941, 181502941, 181502941, 181502941] [-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800] [6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842] [21235.783, 21235.783, 21235.783, 21235.783, 21235.783] [-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891] [58088067, 58088067, 58088067, 58088067, 58088067, 58088067] [-66079723, -66079723, -66079723, -66079723, -66079723, -66079723] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12] [2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19] [2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [63, 63, 63, 63, 63, 63] [11399, 11399, 11399, 11399, 11399, 11399] [-583523026, -583523026, -583523026, -583523026, -583523026, -583523026] [8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293] [-8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906] [23243.16, 23243.16, 23243.16, 23243.16, 23243.16] [716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115] [-40335903, -40335903, -40335903, -40335903, -40335903, -40335903] [74087997, 74087997, 74087997, 74087997, 74087997, 74087997] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27] [2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28] [2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47] ["z", "z", "z", "z", "z", "z"] ["1", "1", "1", "1", "1", "1"] ["xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +8 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [0, 0, 0, 0, 0, 0] [-43, -43, -43, -43, -43, -43] [13560, 13560, 13560, 13560, 13560, 13560] [-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513] [7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769] [-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413] [12225.427, 12225.427, 12225.427, 12225.427, 12225.427] [-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191] [-35959452, -35959452, -35959452, -35959452, -35959452, -35959452] [-4256846, -4256846, -4256846, -4256846, -4256846, -4256846] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43] [2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05] [2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04] ["V", "V", "V", "V", "V", "V"] ["a", "a", "a", "a", "a", "a"] ["8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [1, 1, 1, 1, 1, 1] [-7, -7, -7, -7, -7, -7] [18655, 18655, 18655, 18655, 18655, 18655] [6240356, 6240356, 6240356, 6240356, 6240356, 6240356] [4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393] [323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978] [2972.2478, 2972.2478, 2972.2478, 2972.2478, 2972.2478] [-1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008] [-84949097, -84949097, -84949097, -84949097, -84949097, -84949097] [29815991, 29815991, 29815991, 29815991, 29815991, 29815991] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11] [2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10] [2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07] ["9", "9", "9", "9", "9", "9"] ["n", "n", "n", "n", "n", "n"] ["s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +17 [1, 1, 1, 1, 1, 1] [126, 126, 126, 126, 126, 126] [28165, 28165, 28165, 28165, 28165, 28165] [2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721] [-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879] [-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876] [-21681.223, -21681.223, -21681.223, -21681.223, -21681.223] [95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678] [89062729, 89062729, 89062729, 89062729, 89062729, 89062729] [-2464406, -2464406, -2464406, -2464406, -2464406, -2464406] [2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16] [2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36] [2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20] [2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +27 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [18307, 18307, 18307, 18307, 18307, 18307] [76399879, 76399879, 76399879, 76399879, 76399879, 76399879] [8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699] [2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999] [-16678.924, -16678.924, -16678.924, -16678.924, -16678.924] [-2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261] [12639967, 12639967, 12639967, 12639967, 12639967, 12639967] [24677022, 24677022, 24677022, 24677022, 24677022, 24677022] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55] [2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31] [2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44] ["d", "d", "d", "d", "d", "d"] ["a", "a", "a", "a", "a", "a"] ["dGkS", "dGkS", "dGkS", "dGkS", "dGkS", "dGkS"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +41 [0, 0, 0, 0, 0, 0] [-25, -25, -25, -25, -25, -25] [28704, 28704, 28704, 28704, 28704, 28704] [-437867812, -437867812, -437867812, -437867812, -437867812, -437867812] [5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079] [462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597] [13629.614, 13629.614, 13629.614, 13629.614, 13629.614] [213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962] [86704879, 86704879, 86704879, 86704879, 86704879, 86704879] [-2662959, -2662959, -2662959, -2662959, -2662959, -2662959] [2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04] [2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08] [2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13] [2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07] ["4", "4", "4", "4", "4", "4"] ["a", "a", "a", "a", "a", "a"] ["2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +44 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, 61] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +48 [0, 0, 0, 0, 0, 0] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +49 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +52 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [-6985, -6985, -6985, -6985, -6985, -6985] [826683531, 826683531, 826683531, 826683531, 826683531, 826683531] [-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361] [4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446] [-6490.2471, -6490.2471, -6490.2471, -6490.2471, -6490.2471] [1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261] [-25002204, -25002204, -25002204, -25002204, -25002204, -25002204] [-97805693, -97805693, -97805693, -97805693, -97805693, -97805693] [2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02] [2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22] [2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19] [2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09] ["p", "p", "p", "p", "p", "p"] ["a", "a", "a", "a", "a", "a"] ["WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +55 [1, 1, 1, 1, 1, 1] [65, 65, 65, 65, 65, 65] [18805, 18805, 18805, 18805, 18805, 18805] [229725878, 229725878, 229725878, 229725878, 229725878, 229725878] [2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325] [5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371] [12354.624, 12354.624, 12354.624, 12354.624, 12354.624] [1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771] [20409908, 20409908, 20409908, 20409908, 20409908, 20409908] [-69625379, -69625379, -69625379, -69625379, -69625379, -69625379] [2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09] [2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54] [2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19] [2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +61 [1, 1, 1, 1, 1, 1] [121, 121, 121, 121, 121, 121] [31806, 31806, 31806, 31806, 31806, 31806] [-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562] [-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717] [4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431] [10719.892, 10719.892, 10719.892, 10719.892, 10719.892] [1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421] [78876206, 78876206, 78876206, 78876206, 78876206, 78876206] [-77837482, -77837482, -77837482, -77837482, -77837482, -77837482] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07] [2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15] [2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +62 [0, 0, 0, 0, 0, 0] [-126, -126, -126, -126, -126, -126] [-3813, -3813, -3813, -3813, -3813, -3813] [1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540] [-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217] [5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397] [-17121.195, -17121.195, -17121.195, -17121.195, -17121.195] [-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562] [9626406, 9626406, 9626406, 9626406, 9626406, 9626406] [-8723076, -8723076, -8723076, -8723076, -8723076, -8723076] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20] [2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06] [2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10] ["B", "B", "B", "B", "B", "B"] ["a", "a", "a", "a", "a", "a"] ["8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +76 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +84 [0, 0, 0, 0, 0, 0] [-10, -10, -10, -10, -10, -10] [9493, 9493, 9493, 9493, 9493, 9493] [-547874696, -547874696, -547874696, -547874696, -547874696, -547874696] [-115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756] [4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085] [13718.372, 13718.372, 13718.372, 13718.372, 13718.372] [-978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697] [-88516589, -88516589, -88516589, -88516589, -88516589, -88516589] [-62683124, -62683124, -62683124, -62683124, -62683124, -62683124] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43] [2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25] [2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14] ["J", "J", "J", "J", "J", "J"] ["a", "a", "a", "a", "a", "a"] ["yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +88 [1, 1, 1, 1, 1, 1] [23, 23, 23, 23, 23, 23] [14833, 14833, 14833, 14833, 14833, 14833] [2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015] [7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930] [-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347] [-29817.77, -29817.77, -29817.77, -29817.77, -29817.77] [-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042] [44414137, 44414137, 44414137, 44414137, 44414137, 44414137] [75659236, 75659236, 75659236, 75659236, 75659236, 75659236] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19] [2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27] [2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51] ["a", "a", "a", "a", "a", "a"] ["a", "a", "a", "a", "a", "a"] ["gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_2pc -- + +-- !sql_2pc_abort -- + +-- !sql_2pc_commit -- +2 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +3 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [-67, -67, -67, -67, -67, -67] [-30372, -30372, -30372, -30372, -30372, -30372] [181502941, 181502941, 181502941, 181502941, 181502941, 181502941] [-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800] [6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842] [21235.783, 21235.783, 21235.783, 21235.783, 21235.783] [-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891] [58088067, 58088067, 58088067, 58088067, 58088067, 58088067] [-66079723, -66079723, -66079723, -66079723, -66079723, -66079723] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12] [2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19] [2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +8 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [0, 0, 0, 0, 0, 0] [-43, -43, -43, -43, -43, -43] [13560, 13560, 13560, 13560, 13560, 13560] [-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513] [7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769] [-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413] [12225.427, 12225.427, 12225.427, 12225.427, 12225.427] [-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191] [-35959452, -35959452, -35959452, -35959452, -35959452, -35959452] [-4256846, -4256846, -4256846, -4256846, -4256846, -4256846] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43] [2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05] [2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04] ["V", "V", "V", "V", "V", "V"] ["a", "a", "a", "a", "a", "a"] ["8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +17 [1, 1, 1, 1, 1, 1] [126, 126, 126, 126, 126, 126] [28165, 28165, 28165, 28165, 28165, 28165] [2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721] [-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879] [-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876] [-21681.223, -21681.223, -21681.223, -21681.223, -21681.223] [95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678] [89062729, 89062729, 89062729, 89062729, 89062729, 89062729] [-2464406, -2464406, -2464406, -2464406, -2464406, -2464406] [2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16] [2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36] [2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20] [2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +27 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [18307, 18307, 18307, 18307, 18307, 18307] [76399879, 76399879, 76399879, 76399879, 76399879, 76399879] [8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699] [2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999] [-16678.924, -16678.924, -16678.924, -16678.924, -16678.924] [-2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261] [12639967, 12639967, 12639967, 12639967, 12639967, 12639967] [24677022, 24677022, 24677022, 24677022, 24677022, 24677022] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55] [2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31] [2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44] ["d", "d", "d", "d", "d", "d"] ["a", "a", "a", "a", "a", "a"] ["dGkS", "dGkS", "dGkS", "dGkS", "dGkS", "dGkS"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +41 [0, 0, 0, 0, 0, 0] [-25, -25, -25, -25, -25, -25] [28704, 28704, 28704, 28704, 28704, 28704] [-437867812, -437867812, -437867812, -437867812, -437867812, -437867812] [5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079] [462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597] [13629.614, 13629.614, 13629.614, 13629.614, 13629.614] [213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962] [86704879, 86704879, 86704879, 86704879, 86704879, 86704879] [-2662959, -2662959, -2662959, -2662959, -2662959, -2662959] [2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04] [2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08] [2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13] [2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07] ["4", "4", "4", "4", "4", "4"] ["a", "a", "a", "a", "a", "a"] ["2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +44 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, 61] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +48 [0, 0, 0, 0, 0, 0] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +49 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +52 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [-6985, -6985, -6985, -6985, -6985, -6985] [826683531, 826683531, 826683531, 826683531, 826683531, 826683531] [-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361] [4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446] [-6490.2471, -6490.2471, -6490.2471, -6490.2471, -6490.2471] [1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261] [-25002204, -25002204, -25002204, -25002204, -25002204, -25002204] [-97805693, -97805693, -97805693, -97805693, -97805693, -97805693] [2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02] [2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22] [2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19] [2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09] ["p", "p", "p", "p", "p", "p"] ["a", "a", "a", "a", "a", "a"] ["WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +55 [1, 1, 1, 1, 1, 1] [65, 65, 65, 65, 65, 65] [18805, 18805, 18805, 18805, 18805, 18805] [229725878, 229725878, 229725878, 229725878, 229725878, 229725878] [2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325] [5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371] [12354.624, 12354.624, 12354.624, 12354.624, 12354.624] [1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771] [20409908, 20409908, 20409908, 20409908, 20409908, 20409908] [-69625379, -69625379, -69625379, -69625379, -69625379, -69625379] [2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09] [2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54] [2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19] [2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +61 [1, 1, 1, 1, 1, 1] [121, 121, 121, 121, 121, 121] [31806, 31806, 31806, 31806, 31806, 31806] [-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562] [-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717] [4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431] [10719.892, 10719.892, 10719.892, 10719.892, 10719.892] [1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421] [78876206, 78876206, 78876206, 78876206, 78876206, 78876206] [-77837482, -77837482, -77837482, -77837482, -77837482, -77837482] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07] [2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15] [2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +62 [0, 0, 0, 0, 0, 0] [-126, -126, -126, -126, -126, -126] [-3813, -3813, -3813, -3813, -3813, -3813] [1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540] [-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217] [5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397] [-17121.195, -17121.195, -17121.195, -17121.195, -17121.195] [-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562] [9626406, 9626406, 9626406, 9626406, 9626406, 9626406] [-8723076, -8723076, -8723076, -8723076, -8723076, -8723076] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20] [2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06] [2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10] ["B", "B", "B", "B", "B", "B"] ["a", "a", "a", "a", "a", "a"] ["8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +76 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +84 [0, 0, 0, 0, 0, 0] [-10, -10, -10, -10, -10, -10] [9493, 9493, 9493, 9493, 9493, 9493] [-547874696, -547874696, -547874696, -547874696, -547874696, -547874696] [-115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756] [4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085] [13718.372, 13718.372, 13718.372, 13718.372, 13718.372] [-978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697] [-88516589, -88516589, -88516589, -88516589, -88516589, -88516589] [-62683124, -62683124, -62683124, -62683124, -62683124, -62683124] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43] [2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25] [2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14] ["J", "J", "J", "J", "J", "J"] ["a", "a", "a", "a", "a", "a"] ["yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +88 [1, 1, 1, 1, 1, 1] [23, 23, 23, 23, 23, 23] [14833, 14833, 14833, 14833, 14833, 14833] [2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015] [7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930] [-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347] [-29817.77, -29817.77, -29817.77, -29817.77, -29817.77] [-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042] [44414137, 44414137, 44414137, 44414137, 44414137, 44414137] [75659236, 75659236, 75659236, 75659236, 75659236, 75659236] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19] [2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27] [2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51] ["a", "a", "a", "a", "a", "a"] ["a", "a", "a", "a", "a", "a"] ["gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_2pc -- + +-- !sql_2pc_abort -- + +-- !sql_2pc_commit -- +2 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +3 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [-67, -67, -67, -67, -67, -67] [-30372, -30372, -30372, -30372, -30372, -30372] [181502941, 181502941, 181502941, 181502941, 181502941, 181502941] [-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800] [6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842] [21235.783, 21235.783, 21235.783, 21235.783, 21235.783] [-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891] [58088067, 58088067, 58088067, 58088067, 58088067, 58088067] [-66079723, -66079723, -66079723, -66079723, -66079723, -66079723] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12] [2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19] [2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +8 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [0, 0, 0, 0, 0, 0] [-43, -43, -43, -43, -43, -43] [13560, 13560, 13560, 13560, 13560, 13560] [-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513] [7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769] [-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413] [12225.427, 12225.427, 12225.427, 12225.427, 12225.427] [-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191] [-35959452, -35959452, -35959452, -35959452, -35959452, -35959452] [-4256846, -4256846, -4256846, -4256846, -4256846, -4256846] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43] [2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05] [2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04] ["V", "V", "V", "V", "V", "V"] ["a", "a", "a", "a", "a", "a"] ["8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +17 [1, 1, 1, 1, 1, 1] [126, 126, 126, 126, 126, 126] [28165, 28165, 28165, 28165, 28165, 28165] [2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721] [-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879] [-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876] [-21681.223, -21681.223, -21681.223, -21681.223, -21681.223] [95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678] [89062729, 89062729, 89062729, 89062729, 89062729, 89062729] [-2464406, -2464406, -2464406, -2464406, -2464406, -2464406] [2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16] [2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36] [2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20] [2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +27 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [18307, 18307, 18307, 18307, 18307, 18307] [76399879, 76399879, 76399879, 76399879, 76399879, 76399879] [8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699] [2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999] [-16678.924, -16678.924, -16678.924, -16678.924, -16678.924] [-2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261] [12639967, 12639967, 12639967, 12639967, 12639967, 12639967] [24677022, 24677022, 24677022, 24677022, 24677022, 24677022] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55] [2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31] [2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44] ["d", "d", "d", "d", "d", "d"] ["a", "a", "a", "a", "a", "a"] ["dGkS", "dGkS", "dGkS", "dGkS", "dGkS", "dGkS"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +41 [0, 0, 0, 0, 0, 0] [-25, -25, -25, -25, -25, -25] [28704, 28704, 28704, 28704, 28704, 28704] [-437867812, -437867812, -437867812, -437867812, -437867812, -437867812] [5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079] [462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597] [13629.614, 13629.614, 13629.614, 13629.614, 13629.614] [213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962] [86704879, 86704879, 86704879, 86704879, 86704879, 86704879] [-2662959, -2662959, -2662959, -2662959, -2662959, -2662959] [2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04] [2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08] [2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13] [2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07] ["4", "4", "4", "4", "4", "4"] ["a", "a", "a", "a", "a", "a"] ["2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +44 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, 61] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +48 [0, 0, 0, 0, 0, 0] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +49 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +52 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [-6985, -6985, -6985, -6985, -6985, -6985] [826683531, 826683531, 826683531, 826683531, 826683531, 826683531] [-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361] [4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446] [-6490.2471, -6490.2471, -6490.2471, -6490.2471, -6490.2471] [1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261] [-25002204, -25002204, -25002204, -25002204, -25002204, -25002204] [-97805693, -97805693, -97805693, -97805693, -97805693, -97805693] [2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02] [2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22] [2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19] [2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09] ["p", "p", "p", "p", "p", "p"] ["a", "a", "a", "a", "a", "a"] ["WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +55 [1, 1, 1, 1, 1, 1] [65, 65, 65, 65, 65, 65] [18805, 18805, 18805, 18805, 18805, 18805] [229725878, 229725878, 229725878, 229725878, 229725878, 229725878] [2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325, 2742856458318615325] [5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371, 5907702768956232371] [12354.624, 12354.624, 12354.624, 12354.624, 12354.624] [1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771, 1697579881.9474771] [20409908, 20409908, 20409908, 20409908, 20409908, 20409908] [-69625379, -69625379, -69625379, -69625379, -69625379, -69625379] [2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09, 2023-08-09] [2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54, 2022-12-17 11:47:54] [2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19, 2023-03-19] [2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52, 2023-03-13 10:31:52] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7", "BU2JJTmOKfXr9q8SjWLRUhxn2Me7HFpkMAJxCJCRvnnUJg2l3zXXOnLavxUNt7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +61 [1, 1, 1, 1, 1, 1] [121, 121, 121, 121, 121, 121] [31806, 31806, 31806, 31806, 31806, 31806] [-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562] [-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717] [4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431] [10719.892, 10719.892, 10719.892, 10719.892, 10719.892] [1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421] [78876206, 78876206, 78876206, 78876206, 78876206, 78876206] [-77837482, -77837482, -77837482, -77837482, -77837482, -77837482] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07] [2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15] [2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +62 [0, 0, 0, 0, 0, 0] [-126, -126, -126, -126, -126, -126] [-3813, -3813, -3813, -3813, -3813, -3813] [1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540] [-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217] [5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397] [-17121.195, -17121.195, -17121.195, -17121.195, -17121.195] [-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562] [9626406, 9626406, 9626406, 9626406, 9626406, 9626406] [-8723076, -8723076, -8723076, -8723076, -8723076, -8723076] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20] [2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06] [2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10] ["B", "B", "B", "B", "B", "B"] ["a", "a", "a", "a", "a", "a"] ["8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +76 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +84 [0, 0, 0, 0, 0, 0] [-10, -10, -10, -10, -10, -10] [9493, 9493, 9493, 9493, 9493, 9493] [-547874696, -547874696, -547874696, -547874696, -547874696, -547874696] [-115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756, -115057683458952756] [4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085, 4473017779279230085] [13718.372, 13718.372, 13718.372, 13718.372, 13718.372] [-978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697, -978213266.02697] [-88516589, -88516589, -88516589, -88516589, -88516589, -88516589] [-62683124, -62683124, -62683124, -62683124, -62683124, -62683124] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43, 2022-12-13 00:33:43] [2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25, 2023-03-25] [2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14, 2022-09-15 10:53:14] ["J", "J", "J", "J", "J", "J"] ["a", "a", "a", "a", "a", "a"] ["yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq", "yQtIngWQxQTtvo1z2xxWMOT6JdaZT599ZTsOKOxwERkicW5YhScbCNrQAz8BHaarqK8AHPwvT2uXRlFKu6uZLIONVqdMR0Irap9OzNVIJCJmkCq"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +88 [1, 1, 1, 1, 1, 1] [23, 23, 23, 23, 23, 23] [14833, 14833, 14833, 14833, 14833, 14833] [2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015] [7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930] [-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347] [-29817.77, -29817.77, -29817.77, -29817.77, -29817.77] [-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042] [44414137, 44414137, 44414137, 44414137, 44414137, 44414137] [75659236, 75659236, 75659236, 75659236, 75659236, 75659236] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19] [2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27] [2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51] ["a", "a", "a", "a", "a", "a"] ["a", "a", "a", "a", "a", "a"] ["gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_skip_lines -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_skip_lines -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_skip_lines -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N + +-- !sql_skip_lines -- +8 2023-08-14 true 109 -31573 -1362465190 3990845741226497177 2732763251146840270 -25698.553 1.312831962567818E9 99999999.9 99999999.9 2023-03-07T14:13:19 2022-10-18 2023-07-16T05:03:13 D PBn1wa6X8WneZYLMac11zzyhGl7tPXB5XgjmOV8L6uav9ja5oY433ktb2yhyQQIqBveZPkme {"animal":"lion","weight":200,"habitat":["savannah","grassland"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +20 2023-08-17 false -5 18158 784479801 1485484354598941738 -6632681928222776815 9708.431 -3.30432620706069E8 -99999999.9 99999999.9 2022-09-15T21:40:55 2023-02-23 2023-08-13T21:31:54 O X 2pYmX2vAhfEEHZZYPsgAmda1G7otnwx5TmUC879FPhDeIjvWI79ksBZpfFG2gp7jhCSbpZiecKGklB5SvG8tm31i5SUqe1xrWgLt4HSq7lMJWp75tx2kxD7pRIOpn {"name":"Sarah","age":30,"city":"London","isMarried":false} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +21 2023-08-18 false 63 -27847 -35409596 8638201997392767650 4919963231735304178 -23382.541 -1.803403621426313E9 -22009767.0 99999999.9 2023-03-31T10:56:14 2023-01-20 2023-02-18T13:37:52 N T PSiFwUEx3eVFNtjlnQ70YkgZNvKrGmQ2DN5K9yYHiSdFWeEDB1UpL3Frt8z1kEAIWRDWqXZuyi {"city":"Sydney","population":5312000,"area":2058.7} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +31 2023-08-27 false 17 -18849 1728109133 3266501886640700374 527195452623418935 -24062.328 -1.514348021262435E9 -99999999.9 -99999999.9 2022-10-07T03:24:23 2022-09-25 \N 0 8 yKMiAntORoRa8svnMfcxlOPwwND1m5s2fdS26Xu6cfs6HK5SAibqIp9h8sZcpjHy4 {"team":"Manchester United","players":["Ronaldo","Rooney","Giggs"],"coach":"Ole Gunnar Solskjaer"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +41 2023-08-27 true -104 22750 \N 8527773271030840740 5554497317268279215 -5296.828 -1.71564688801304E9 -99999999.9 99999999.9 2022-12-02T17:56:44 2022-10-12 2023-02-19T07:02:54 V \N E9GzQdTwX1ITUQz27IVznAs6Ca4WwprKk6Odjs6SH75D2F1089QiY3HQ52LXRD1V6xAWjhLE2hWgW3EdHuAOnUDVrb5V {"food":"Sushi","price":10,"restaurant":"Sushi King"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +50 2023-08-06 true 109 -6330 1479023892 -8630800697573159428 -1645095773540208759 17880.96 -1.453844792013949E9 -99999999.9 -99999999.9 2022-09-22T02:03:21 2023-05-14 2023-03-25T02:18:34 m JKnIgXvGVidGiWl9YRSi3mFI7wHKt1sBpWSadKF8VX3LAuElm4sdc9gtxREaUr57oikSYlU8We8h1MWqQlYNiJObl {"city":"Tokyo","temperature":20.5,"humidity":75} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +50 2023-08-24 true 15 14403 \N -6418906115745394180 9205303779366462513 -4331.549 -6.15112179557648E8 99999999.9 -99999999.9 2022-12-29T02:27:20 2023-06-01 2023-08-12T04:50:04 a eCl38sztIvBQvGvGKyYZmyMXy9vIJx197iu3JwP9doJGcrYUl9Uova0rz4iCCgrjlAiZU18Fs9YtCq830nhM {"band":"The Beatles","members":["John Lennon","Paul McCartney","George Harrison","Ringo Starr"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +58 2023-08-22 \N 0 -18231 1832867360 6997858407575297145 2480714305422728023 -5450.489 1.475901032138386E9 -99999999.9 -99999999.9 2023-02-02T05:13:24 2022-09-18 2023-04-23T10:51:15 k LdFXF7Kmfzgmnn2R6zLsXdmi3A2cLBLq4G4WDVNDhxvH7dYH8Kga2WA47uSIxp6NSrwPSdw0ssB1TS8RFJTDJAB0Uba3e05NL2Aiw0ja {"restaurant":"Pizza Hut","menu":["pizza","pasta","salad"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +60 2023-08-27 false -52 -2338 -757056972 1047567408607120856 6541476642780646552 6614.0894 -1.204448798517855E9 99999999.9 99999999.9 2022-12-29T14:47:30 2022-09-24 2023-08-01T12:41:59 O F RM4F1Ke7lkcnuxF2nK0j9VBW3MDcgyHR4pseBjtFnqS6GUkVFuzF6u3Cp9Nv7ab0O6UYrpP4DhU {"game":"Chess","players":2,"time":"1 hour"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +62 2023-08-21 false 81 20302 -200761532 6365479976421007608 \N -29916.533 1.709141750828478E9 99999999.9 -99999999.9 2023-05-04T01:14:51 2022-09-17 2022-12-04T19:30:09 d v BKWy9dTNg1aZW7ancEJAmEDOPK5TwFsNSHbI78emu9gymeIlx5NoLmyii0QAqdzRvSQPZKiqKkwInGCTIBnK1yYkK7zD {"username":"user123","password":"pass123","email":"user123@example.com"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +65 2023-08-09 false 94 31514 814994517 -297697460695940343 734910652450318597 -13061.892 6.2750847041706E7 -9808654.0 \N 2023-08-14T22:01:27 2023-05-19 2022-11-13T13:44:28 V aGeMsI24O12chGlP5ak0AHghAz7bu5MargJBStHnt0yMnChH0JnfYhsfH1u59XIHkJKMsHYktBqORkGlovu8V47E74KeFpaqxn5yLyXfDbhhzUKf {"language":"Python","version":3.9,"frameworks":["Django","Flask"]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +66 2023-08-15 true -91 28378 609923317 4872185586197131212 1207709464099378591 \N -1.863683325985123E9 -99999999.9 -99999999.9 2022-09-24T10:39:23 2022-09-24 2022-10-16T18:36:43 Y z AI1BSPQdKiHJiQH1kguyLSWsDXkC7zwy7PwgWnyGSaa9tBKRex8vHBdxg2QSKZKL2mV2lHz7iI1PnsTd4MXDcIKhqiHyPuQPt2tEtgt0UgF6 {"book":{"title":"The Great Gatsby","author":"F. Scott Fitzgerald"},"year":1925} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +68 2023-08-23 true -73 20117 1737338128 795638676048937749 -5551546237562433901 -30627.04 6.8589475684545E7 99999999.9 99999999.9 2022-12-28T20:26:51 2022-10-04 2023-07-30T00:20:06 y keZ3JlWWpdnPBejf0cuiCQCVBBTd5gjvO08NVdcAFewqL7nRT4N9lnvSU6pWmletA5VbPQCeQapJdcnQCHfZUDCf4ulCnczyqr7SGrbGRT0XYcd7iktKM {"country":"Brazil","continent":"South America","population":211049527} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +80 2023-08-18 false -18 -8971 679027874 6535956962935330265 3960889045799757165 -13219.76 1.187161924505394E9 -99999999.9 -99999999.9 2023-03-11T07:40 2022-11-29 2023-01-14T07:24:07 \N D 3Nhx6xX1qdwaq7lxwLRSKMtJFbC03swWv12mpySSVysH3igGZTiGPuKMsYW7HAkf6CWc7c0nzqDsjuH3FYVMNCWRmfxMrmY8rykQCC4Ve {"car":"BMW","model":"X5","year":2020,"color":"black"} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +81 2023-08-23 false 106 11492 -667795397 4480250461471356146 -5346660566234294101 9082.75 3.85167225902608E8 -99999999.9 99999999.9 2023-03-20T03:33:16 2022-11-24 2023-02-16T18:29:41 G 9 Lk3eNVQNjucbekD1rZmUlGPiXS5JvcWr2LQzRU8GSGIbSag {"flower":"rose","color":"red","fragrance":true} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +85 2023-08-11 true -7 24304 -2043877415 -2024144417867729183 \N 5363.0244 -5.78615669042831E8 -99999999.9 -99999999.9 2023-07-15T01:07:41 2023-08-13 2023-01-20T11:57:48 i WQ9dh9ajPu0y {"country":"France","capital":"Paris","population":67081000} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +90 2023-08-27 true 22 16456 -1476824962 -3279894870153540825 8990195191470116763 26651.906 2.06860148942546E8 -99999999.9 -99999999.9 2022-10-07T03:11:03 2023-03-18 2023-04-15T00:38:33 T L QW0GQ3GoMtHgxPQOWGfVaveynahNpsNs09siMFA1OtO6QEDBQTdivmGyq7bFzejAqwbbVQQpREAmeLjcFSXLnQuou2KbwYD {"company":"Apple","products":[{"name":"iPhone","price":1000},{"name":"MacBook","price":1500}]} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N +91 2023-08-27 true 90 2465 702240964 6373830997821598984 305860046137409400 15991.356 1.599972327386147E9 -99999999.9 \N 2023-04-26T19:31:10 2023-07-21 \N 2 B7YKYBYT8w0YC926bZ8Yz1VzyiWw2NWDAiTlEoPVyz9AXGti2Npg1FxWqWk4hEaALw0ZBSuiAIPj41lq36g5QRpPmAjNPK {"fruit":"apple","color":"red","qty":5,"price":2.5} \N \N \N true 1 2 3 4 5 6.0 7.0 888888888 999999999 2023-08-24 2023-08-24T12:00 2023-08-24 2023-08-24T12:00 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 我能吞下玻璃而不伤身体 \N \N \N \N + +-- !sql_skip_lines -- +2 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +3 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [-67, -67, -67, -67, -67, -67] [-30372, -30372, -30372, -30372, -30372, -30372] [181502941, 181502941, 181502941, 181502941, 181502941, 181502941] [-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800] [6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842] [21235.783, 21235.783, 21235.783, 21235.783, 21235.783] [-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891] [58088067, 58088067, 58088067, 58088067, 58088067, 58088067] [-66079723, -66079723, -66079723, -66079723, -66079723, -66079723] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12] [2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19] [2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [63, 63, 63, 63, 63, 63] [11399, 11399, 11399, 11399, 11399, 11399] [-583523026, -583523026, -583523026, -583523026, -583523026, -583523026] [8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293, 8801164674137231293] [-8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906, -8287675635310193906] [23243.16, 23243.16, 23243.16, 23243.16, 23243.16] [716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115, 716719993.249115] [-40335903, -40335903, -40335903, -40335903, -40335903, -40335903] [74087997, 74087997, 74087997, 74087997, 74087997, 74087997] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27, 2023-08-06 19:03:27] [2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28, 2023-03-28] [2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47, 2022-09-27 09:19:47] ["z", "z", "z", "z", "z", "z"] ["1", "1", "1", "1", "1", "1"] ["xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz", "xdc5RcpnOAqeK6Hmz"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +8 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [0, 0, 0, 0, 0, 0] [-43, -43, -43, -43, -43, -43] [13560, 13560, 13560, 13560, 13560, 13560] [-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513] [7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769] [-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413] [12225.427, 12225.427, 12225.427, 12225.427, 12225.427] [-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191] [-35959452, -35959452, -35959452, -35959452, -35959452, -35959452] [-4256846, -4256846, -4256846, -4256846, -4256846, -4256846] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43] [2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05] [2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04] ["V", "V", "V", "V", "V", "V"] ["a", "a", "a", "a", "a", "a"] ["8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [1, 1, 1, 1, 1, 1] [-7, -7, -7, -7, -7, -7] [18655, 18655, 18655, 18655, 18655, 18655] [6240356, 6240356, 6240356, 6240356, 6240356, 6240356] [4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393, 4552179257266841393] [323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978, 323868824766329978] [2972.2478, 2972.2478, 2972.2478, 2972.2478, 2972.2478] [-1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008, -1177167334.995008] [-84949097, -84949097, -84949097, -84949097, -84949097, -84949097] [29815991, 29815991, 29815991, 29815991, 29815991, 29815991] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11, 2022-09-10 01:11:11] [2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10, 2022-09-10] [2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07, 2023-06-06 20:38:07] ["9", "9", "9", "9", "9", "9"] ["n", "n", "n", "n", "n", "n"] ["s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX", "s093vhFomjJOSwwn3Dii8iK9wevxUFwRUVJE6GAGOZEErIMLY68bUYV1h5d57QWfp7AxBkTbxPqbEh4ZR2Z3wROrAFQjpUtN4Y9Y8H4nC5xj14n5hXqCdJYpzkX"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +17 [1, 1, 1, 1, 1, 1] [126, 126, 126, 126, 126, 126] [28165, 28165, 28165, 28165, 28165, 28165] [2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721] [-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879] [-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876] [-21681.223, -21681.223, -21681.223, -21681.223, -21681.223] [95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678] [89062729, 89062729, 89062729, 89062729, 89062729, 89062729] [-2464406, -2464406, -2464406, -2464406, -2464406, -2464406] [2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16] [2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36] [2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20] [2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +27 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [18307, 18307, 18307, 18307, 18307, 18307] [76399879, 76399879, 76399879, 76399879, 76399879, 76399879] [8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699] [2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999] [-16678.924, -16678.924, -16678.924, -16678.924, -16678.924] [-2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261] [12639967, 12639967, 12639967, 12639967, 12639967, 12639967] [24677022, 24677022, 24677022, 24677022, 24677022, 24677022] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55] [2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31] [2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44] ["d", "d", "d", "d", "d", "d"] ["a", "a", "a", "a", "a", "a"] ["dGkS", "dGkS", "dGkS", "dGkS", "dGkS", "dGkS"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +41 [0, 0, 0, 0, 0, 0] [-25, -25, -25, -25, -25, -25] [28704, 28704, 28704, 28704, 28704, 28704] [-437867812, -437867812, -437867812, -437867812, -437867812, -437867812] [5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079] [462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597] [13629.614, 13629.614, 13629.614, 13629.614, 13629.614] [213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962] [86704879, 86704879, 86704879, 86704879, 86704879, 86704879] [-2662959, -2662959, -2662959, -2662959, -2662959, -2662959] [2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04] [2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08] [2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13] [2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07] ["4", "4", "4", "4", "4", "4"] ["a", "a", "a", "a", "a", "a"] ["2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +44 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, 61] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +48 [0, 0, 0, 0, 0, 0] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +49 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +52 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [-6985, -6985, -6985, -6985, -6985, -6985] [826683531, 826683531, 826683531, 826683531, 826683531, 826683531] [-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361] [4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446] [-6490.2471, -6490.2471, -6490.2471, -6490.2471, -6490.2471] [1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261] [-25002204, -25002204, -25002204, -25002204, -25002204, -25002204] [-97805693, -97805693, -97805693, -97805693, -97805693, -97805693] [2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02] [2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22] [2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19] [2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09] ["p", "p", "p", "p", "p", "p"] ["a", "a", "a", "a", "a", "a"] ["WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +61 [1, 1, 1, 1, 1, 1] [121, 121, 121, 121, 121, 121] [31806, 31806, 31806, 31806, 31806, 31806] [-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562] [-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717] [4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431] [10719.892, 10719.892, 10719.892, 10719.892, 10719.892] [1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421] [78876206, 78876206, 78876206, 78876206, 78876206, 78876206] [-77837482, -77837482, -77837482, -77837482, -77837482, -77837482] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07] [2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15] [2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +62 [0, 0, 0, 0, 0, 0] [-126, -126, -126, -126, -126, -126] [-3813, -3813, -3813, -3813, -3813, -3813] [1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540] [-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217] [5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397] [-17121.195, -17121.195, -17121.195, -17121.195, -17121.195] [-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562] [9626406, 9626406, 9626406, 9626406, 9626406, 9626406] [-8723076, -8723076, -8723076, -8723076, -8723076, -8723076] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20] [2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06] [2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10] ["B", "B", "B", "B", "B", "B"] ["a", "a", "a", "a", "a", "a"] ["8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +76 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +88 [1, 1, 1, 1, 1, 1] [23, 23, 23, 23, 23, 23] [14833, 14833, 14833, 14833, 14833, 14833] [2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015] [7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930] [-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347] [-29817.77, -29817.77, -29817.77, -29817.77, -29817.77] [-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042] [44414137, 44414137, 44414137, 44414137, 44414137, 44414137] [75659236, 75659236, 75659236, 75659236, 75659236, 75659236] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19] [2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27] [2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51] ["a", "a", "a", "a", "a", "a"] ["a", "a", "a", "a", "a", "a"] ["gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_skip_lines -- +2 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +3 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [-67, -67, -67, -67, -67, -67] [-30372, -30372, -30372, -30372, -30372, -30372] [181502941, 181502941, 181502941, 181502941, 181502941, 181502941] [-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800] [6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842] [21235.783, 21235.783, 21235.783, 21235.783, 21235.783] [-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891] [58088067, 58088067, 58088067, 58088067, 58088067, 58088067] [-66079723, -66079723, -66079723, -66079723, -66079723, -66079723] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12] [2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19] [2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +8 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [0, 0, 0, 0, 0, 0] [-43, -43, -43, -43, -43, -43] [13560, 13560, 13560, 13560, 13560, 13560] [-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513] [7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769] [-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413] [12225.427, 12225.427, 12225.427, 12225.427, 12225.427] [-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191] [-35959452, -35959452, -35959452, -35959452, -35959452, -35959452] [-4256846, -4256846, -4256846, -4256846, -4256846, -4256846] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43] [2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05] [2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04] ["V", "V", "V", "V", "V", "V"] ["a", "a", "a", "a", "a", "a"] ["8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +17 [1, 1, 1, 1, 1, 1] [126, 126, 126, 126, 126, 126] [28165, 28165, 28165, 28165, 28165, 28165] [2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721] [-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879] [-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876] [-21681.223, -21681.223, -21681.223, -21681.223, -21681.223] [95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678] [89062729, 89062729, 89062729, 89062729, 89062729, 89062729] [-2464406, -2464406, -2464406, -2464406, -2464406, -2464406] [2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16] [2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36] [2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20] [2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +27 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [18307, 18307, 18307, 18307, 18307, 18307] [76399879, 76399879, 76399879, 76399879, 76399879, 76399879] [8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699] [2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999] [-16678.924, -16678.924, -16678.924, -16678.924, -16678.924] [-2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261] [12639967, 12639967, 12639967, 12639967, 12639967, 12639967] [24677022, 24677022, 24677022, 24677022, 24677022, 24677022] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55] [2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31] [2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44] ["d", "d", "d", "d", "d", "d"] ["a", "a", "a", "a", "a", "a"] ["dGkS", "dGkS", "dGkS", "dGkS", "dGkS", "dGkS"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +41 [0, 0, 0, 0, 0, 0] [-25, -25, -25, -25, -25, -25] [28704, 28704, 28704, 28704, 28704, 28704] [-437867812, -437867812, -437867812, -437867812, -437867812, -437867812] [5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079] [462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597] [13629.614, 13629.614, 13629.614, 13629.614, 13629.614] [213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962] [86704879, 86704879, 86704879, 86704879, 86704879, 86704879] [-2662959, -2662959, -2662959, -2662959, -2662959, -2662959] [2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04] [2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08] [2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13] [2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07] ["4", "4", "4", "4", "4", "4"] ["a", "a", "a", "a", "a", "a"] ["2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +44 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, 61] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +48 [0, 0, 0, 0, 0, 0] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +49 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +52 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [-6985, -6985, -6985, -6985, -6985, -6985] [826683531, 826683531, 826683531, 826683531, 826683531, 826683531] [-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361] [4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446] [-6490.2471, -6490.2471, -6490.2471, -6490.2471, -6490.2471] [1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261] [-25002204, -25002204, -25002204, -25002204, -25002204, -25002204] [-97805693, -97805693, -97805693, -97805693, -97805693, -97805693] [2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02] [2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22] [2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19] [2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09] ["p", "p", "p", "p", "p", "p"] ["a", "a", "a", "a", "a", "a"] ["WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +61 [1, 1, 1, 1, 1, 1] [121, 121, 121, 121, 121, 121] [31806, 31806, 31806, 31806, 31806, 31806] [-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562] [-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717] [4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431] [10719.892, 10719.892, 10719.892, 10719.892, 10719.892] [1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421] [78876206, 78876206, 78876206, 78876206, 78876206, 78876206] [-77837482, -77837482, -77837482, -77837482, -77837482, -77837482] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07] [2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15] [2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +62 [0, 0, 0, 0, 0, 0] [-126, -126, -126, -126, -126, -126] [-3813, -3813, -3813, -3813, -3813, -3813] [1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540] [-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217] [5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397] [-17121.195, -17121.195, -17121.195, -17121.195, -17121.195] [-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562] [9626406, 9626406, 9626406, 9626406, 9626406, 9626406] [-8723076, -8723076, -8723076, -8723076, -8723076, -8723076] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20] [2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06] [2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10] ["B", "B", "B", "B", "B", "B"] ["a", "a", "a", "a", "a", "a"] ["8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +76 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +88 [1, 1, 1, 1, 1, 1] [23, 23, 23, 23, 23, 23] [14833, 14833, 14833, 14833, 14833, 14833] [2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015] [7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930] [-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347] [-29817.77, -29817.77, -29817.77, -29817.77, -29817.77] [-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042] [44414137, 44414137, 44414137, 44414137, 44414137, 44414137] [75659236, 75659236, 75659236, 75659236, 75659236, 75659236] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19] [2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27] [2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51] ["a", "a", "a", "a", "a", "a"] ["a", "a", "a", "a", "a", "a"] ["gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + +-- !sql_skip_lines -- +2 [0, 0, 0, 0, 0, 0] [117, 117, 117, 117, 117, 117] [-4744, -4744, -4744, -4744, -4744, -4744] [-1593211961, -1593211961, -1593211961, -1593211961, -1593211961, -1593211961] [-3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780, -3869640069299678780] [8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567, 8491817458398170567] [-30948.857, -30948.857, -30948.857, -30948.857, -30948.857] [804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905, 804341131.229905] [-74019648, -74019648, -74019648, -74019648, -74019648, -74019648] [13024168, 13024168, 13024168, 13024168, 13024168, 13024168] [2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22, 2023-08-22] [2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12, 2022-09-30 07:47:12] [2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21, 2023-04-21] [2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56, 2022-11-24 15:07:56] ["g", "g", "g", "g", "g", "g"] ["a", "a", "a", "a", "a", "a"] ["S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl", "S9JEYFrLN4zr1vX1yPUE6ovSX431nJdCuttpBUOVMrp844vBfHStO7laHNc5sI9MehAi8GbGDGV3t322DPMy7SBlquU5D7jsGISMNpX4IWbn3Yrsl"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +3 [0, 0, 0, 0, 0, 0] [65, 65, 65, 65, 65, 65] [-4963, -4963, -4963, -4963, -4963, -4963] [-1415431954, -1415431954, -1415431954, -1415431954, -1415431954, -1415431954] [-3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000, -3804309860450207000] [8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376, 8209240008557215376] [-5058.13, -5058.13, -5058.13, -5058.13, -5058.13] [1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352, 1034763010.616352] [3858273, 3858273, 3858273, 3858273, 3858273, 3858273] [-3634150, -3634150, -3634150, -3634150, -3634150, -3634150] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44, 2023-03-06 07:47:44] [2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27, 2023-04-27] [2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11, 2022-12-20 15:40:11] ["i", "i", "i", "i", "i", "i"] ["G", "G", "G", "G", "G", "G"] ["XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ", "XuyX5eljhfMNqMmuOGkFNvyoKFyFMzGWPuGQPxAi6NYV6JA2aooYGJ0CgQ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +4 [0, 0, 0, 0, 0, 0] [-67, -67, -67, -67, -67, -67] [-30372, -30372, -30372, -30372, -30372, -30372] [181502941, 181502941, 181502941, 181502941, 181502941, 181502941] [-2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800, -2062236823576972800] [6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842, 6357002962400127842] [21235.783, 21235.783, 21235.783, 21235.783, 21235.783] [-1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891, -1101694755.713891] [58088067, 58088067, 58088067, 58088067, 58088067, 58088067] [-66079723, -66079723, -66079723, -66079723, -66079723, -66079723] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12, 2023-06-16 18:07:12] [2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19, 2022-11-19] [2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12, 2023-08-23 08:19:12] ["G", "G", "G", "G", "G", "G"] ["a", "a", "a", "a", "a", "a"] ["tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W", "tX4eqSuxvREnF4UIk8OnDyDZM1yT2G2IVzcNB4Lejgxr1W"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +8 [1, 1, 1, 1, 1, 1] [-11, -11, -11, -11, -11, -11] [-9648, -9648, -9648, -9648, -9648, -9648] [-505356927, -505356927, -505356927, -505356927, -505356927, -505356927] [7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037, 7604760670442035037] [1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798, 1634770507625165798] [10822.962, 10822.962, 10822.962, 10822.962, 10822.962] [1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071, 1987551048.863071] [64879544, 64879544, 64879544, 64879544, 64879544, 64879544] [-55896622, -55896622, -55896622, -55896622, -55896622, -55896622] [2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08, 2023-08-08] [2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23, 2023-01-05 00:55:23] [2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16, 2023-06-16] [2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26, 2023-04-06 03:40:26] ["U", "U", "U", "U", "U", "U"] ["G", "G", "G", "G", "G", "G"] ["iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ", "iyAI9214vOaKrPo1EmVesccN0PdeCC0rKXzRJv33KpnnJbG0o0FXubzuBfrYTQZ"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +16 [0, 0, 0, 0, 0, 0] [-43, -43, -43, -43, -43, -43] [13560, 13560, 13560, 13560, 13560, 13560] [-1743686513, -1743686513, -1743686513, -1743686513, -1743686513, -1743686513] [7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769, 7234719406392208769] [-3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413, -3871745630024229413] [12225.427, 12225.427, 12225.427, 12225.427, 12225.427] [-1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191, -1352141342.04191] [-35959452, -35959452, -35959452, -35959452, -35959452, -35959452] [-4256846, -4256846, -4256846, -4256846, -4256846, -4256846] [2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24, 2023-08-24] [2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43, 2022-12-25 20:40:43] [2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05, 2023-01-05] [2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04, 2023-01-27 19:34:04] ["V", "V", "V", "V", "V", "V"] ["a", "a", "a", "a", "a", "a"] ["8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj", "8PRRNG2OCMIFR5k9nuC0O4TDT6hxwj"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +17 [1, 1, 1, 1, 1, 1] [126, 126, 126, 126, 126, 126] [28165, 28165, 28165, 28165, 28165, 28165] [2032059721, 2032059721, 2032059721, 2032059721, 2032059721, 2032059721] [-2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879, -2686776977990574879] [-8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876, -8498149444423102876] [-21681.223, -21681.223, -21681.223, -21681.223, -21681.223] [95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678, 95345732.447678] [89062729, 89062729, 89062729, 89062729, 89062729, 89062729] [-2464406, -2464406, -2464406, -2464406, -2464406, -2464406] [2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16, 2023-08-16] [2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36, 2023-03-05 10:43:36] [2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20, 2023-08-20] [2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44, 2023-04-11 00:33:44] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb", "XJ3RV27MQpsPD30k1xV0RGSuomCT1z5oEfOiF2gNisoMyFhoClXJJ8eIZSeasLKeJwflG4In7xn54n9oI16rpRQJkeb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +27 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [18307, 18307, 18307, 18307, 18307, 18307] [76399879, 76399879, 76399879, 76399879, 76399879, 76399879] [8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699, 8050764818738996699] [2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999, 2402219865213589999] [-16678.924, -16678.924, -16678.924, -16678.924, -16678.924] [-2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261, -2053879544.8447261] [12639967, 12639967, 12639967, 12639967, 12639967, 12639967] [24677022, 24677022, 24677022, 24677022, 24677022, 24677022] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55, 2023-01-09 07:35:55] [2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31, 2023-07-31] [2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44, 2023-03-15 06:40:44] ["d", "d", "d", "d", "d", "d"] ["a", "a", "a", "a", "a", "a"] ["dGkS", "dGkS", "dGkS", "dGkS", "dGkS", "dGkS"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +41 [0, 0, 0, 0, 0, 0] [-25, -25, -25, -25, -25, -25] [28704, 28704, 28704, 28704, 28704, 28704] [-437867812, -437867812, -437867812, -437867812, -437867812, -437867812] [5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079, 5508042206505207079] [462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597, 462527544684407597] [13629.614, 13629.614, 13629.614, 13629.614, 13629.614] [213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962, 213722401.337962] [86704879, 86704879, 86704879, 86704879, 86704879, 86704879] [-2662959, -2662959, -2662959, -2662959, -2662959, -2662959] [2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04, 2023-08-04] [2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08, 2022-12-28 16:55:08] [2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13, 2023-05-13] [2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07, 2023-03-10 18:15:07] ["4", "4", "4", "4", "4", "4"] ["a", "a", "a", "a", "a", "a"] ["2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI", "2dxEKrIfvZTmYHO6jXR7HMuJAJrj1dJD2WRroeHL20dGolyHdcI"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +44 [0, 0, 0, 0, 0, 0] [61, 61, 61, 61, 61, 61] [-23419, -23419, -23419, -23419, -23419, -23419] [378600280, 378600280, 378600280, 378600280, 378600280, 378600280] [6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679, 6788166268039991679] [-2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852, -2814786606977504852] [-20151.432, -20151.432, -20151.432, -20151.432, -20151.432] [-1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291, -1248229001.2182291] [-45191385, -45191385, -45191385, -45191385, -45191385, -45191385] [-27910227, -27910227, -27910227, -27910227, -27910227, -27910227] [2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13, 2023-08-13] [2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32, 2023-05-18 04:25:32] [2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20, 2023-03-20] [2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00, 2022-12-21 11:34:00] ["4", "4", "4", "4", "4", "4"] ["I", "I", "I", "I", "I", "I"] ["NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1", "NZXyjFFg0Yunu6fDXpyr9OlUtMNbaJbSENshFUaeCdMR64vgELMWSxvUxwLfmhzoDY1z6bLCyjuGmMUk9hhnF9hKsFkgpbcPo2nz1"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +48 [0, 0, 0, 0, 0, 0] [88, 88, 88, 88, 88, 88] [-18899, -18899, -18899, -18899, -18899, -18899] [1953750640, 1953750640, 1953750640, 1953750640, 1953750640, 1953750640] [-6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312, -6083034186246180312] [7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662, 7861718260607212662] [20562.791, 20562.791, 20562.791, 20562.791, 20562.791] [-1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959, -1597994654.9033959] [91765768, 91765768, 91765768, 91765768, 91765768, 91765768] [-47488138, -47488138, -47488138, -47488138, -47488138, -47488138] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24, 2022-11-04 07:47:24] [2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16, 2023-07-16] [2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07, 2022-08-29 04:51:07] ["u", "u", "u", "u", "u", "u"] ["P", "P", "P", "P", "P", "P"] ["wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv", "wodMoILg2POwMnYHhY33utnoZ325ocWKXPLvo1Cxx8C2Wj8maoUhfwozaHgjzoOqeW9lMj2nNinNDJV2dnXsNfa0hVeNzonA7s84gYHSyHXDzvjv"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +49 [0, 0, 0, 0, 0, 0] [126, 126, 126, 126, 126, 126] [31661, 31661, 31661, 31661, 31661, 31661] [359703581, 359703581, 359703581, 359703581, 359703581, 359703581] [-2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939, -2399575246807057939] [7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391, 7684667782059034391] [-504.68152, -504.68152, -504.68152, -504.68152, -504.68152] [2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791, 2121528178.4887791] [-99977803, -99977803, -99977803, -99977803, -99977803, -99977803] [25652192, 25652192, 25652192, 25652192, 25652192, 25652192] [2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25, 2023-08-25] [2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11, 2023-07-14 17:16:11] [2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28, 2022-12-28] [2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11, 2023-04-10 04:24:11] ["i", "i", "i", "i", "i", "i"] ["0", "0", "0", "0", "0", "0"] ["X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7", "X53UUdHkzSVyWMmGyI5i6M4ohehRhx1NR02IjJsuKy64Tp0KJvsHgjJ64F2qHOpNQ17EUnIPJlmpCV32vDZkJwXjhhY1eObFH2Ru7gHqsmFCQ2zy7"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +52 [0, 0, 0, 0, 0, 0] [-7, -7, -7, -7, -7, -7] [-6985, -6985, -6985, -6985, -6985, -6985] [826683531, 826683531, 826683531, 826683531, 826683531, 826683531] [-8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361, -8966681855246736361] [4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446, 4814686163176635446] [-6490.2471, -6490.2471, -6490.2471, -6490.2471, -6490.2471] [1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261, 1076976372.0338261] [-25002204, -25002204, -25002204, -25002204, -25002204, -25002204] [-97805693, -97805693, -97805693, -97805693, -97805693, -97805693] [2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02, 2023-08-02] [2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22, 2022-11-21 04:05:22] [2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19, 2023-05-19] [2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09, 2023-03-12 05:07:09] ["p", "p", "p", "p", "p", "p"] ["a", "a", "a", "a", "a", "a"] ["WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p", "WG0vFztUWdoC7f8D14NbhHAK302bHf6s1JKNU2hiIjZ5ABhHwikfSzCAKdr04s6bhGkkssdVzRuSSheQ0rFUGkueuLch5p"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +61 [1, 1, 1, 1, 1, 1] [121, 121, 121, 121, 121, 121] [31806, 31806, 31806, 31806, 31806, 31806] [-1410915562, -1410915562, -1410915562, -1410915562, -1410915562, -1410915562] [-250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717, -250403393155768717] [4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431, 4301573778529723431] [10719.892, 10719.892, 10719.892, 10719.892, 10719.892] [1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421, 1073780599.1272421] [78876206, 78876206, 78876206, 78876206, 78876206, 78876206] [-77837482, -77837482, -77837482, -77837482, -77837482, -77837482] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07, 2023-01-31 15:44:07] [2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15, 2023-03-15] [2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16, 2023-02-06 12:53:16] ["U", "U", "U", "U", "U", "U"] ["a", "a", "a", "a", "a", "a"] ["y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb", "y25ujOZPm64KwGfKXGPMgqaUAdIhi8GtBb"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +62 [0, 0, 0, 0, 0, 0] [-126, -126, -126, -126, -126, -126] [-3813, -3813, -3813, -3813, -3813, -3813] [1950651540, 1950651540, 1950651540, 1950651540, 1950651540, 1950651540] [-7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217, -7509418841468966217] [5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397, 5935288575051629397] [-17121.195, -17121.195, -17121.195, -17121.195, -17121.195] [-987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562, -987496114.899562] [9626406, 9626406, 9626406, 9626406, 9626406, 9626406] [-8723076, -8723076, -8723076, -8723076, -8723076, -8723076] [2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01, 2023-08-01] [2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20, 2023-04-24 18:15:20] [2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06, 2022-10-06] [2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10, 2022-10-14 01:34:10] ["B", "B", "B", "B", "B", "B"] ["a", "a", "a", "a", "a", "a"] ["8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC", "8aRmFSUBC"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +76 [1, 1, 1, 1, 1, 1] [-63, -63, -63, -63, -63, -63] [25799, 25799, 25799, 25799, 25799, 25799] [-1387912656, -1387912656, -1387912656, -1387912656, -1387912656, -1387912656] [8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181, 8967926767558546181] [-3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476, -3537865898119184476] [5311.188, 5311.188, 5311.188, 5311.188, 5311.188] [173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049, 173628749.847049] [-28850258, -28850258, -28850258, -28850258, -28850258, -28850258] [-99202733, -99202733, -99202733, -99202733, -99202733, -99202733] [2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12, 2023-08-12] [2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22, 2023-05-28 15:56:22] [2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11, 2023-06-11] [2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10, 2022-11-29 16:23:10] ["7", "7", "7", "7", "7", "7"] ["j", "j", "j", "j", "j", "j"] ["fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0", "fjbvP5vNgAwYBIAEnONGcVbrBpGM3mqVeBjDQs4KQlLEgNbnHVIIscgaRuPdDjU0"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +88 [1, 1, 1, 1, 1, 1] [23, 23, 23, 23, 23, 23] [14833, 14833, 14833, 14833, 14833, 14833] [2017339015, 2017339015, 2017339015, 2017339015, 2017339015, 2017339015] [7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930, 7437673973249797930] [-5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347, -5508026584283164347] [-29817.77, -29817.77, -29817.77, -29817.77, -29817.77] [-1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042, -1646754251.102042] [44414137, 44414137, 44414137, 44414137, 44414137, 44414137] [75659236, 75659236, 75659236, 75659236, 75659236, 75659236] [2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26, 2023-08-26] [2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19, 2022-12-15 12:25:19] [2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27, 2022-11-27] [2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51, 2023-02-22 00:46:51] ["a", "a", "a", "a", "a", "a"] ["a", "a", "a", "a", "a", "a"] ["gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh", "gfynMkzlxqeETupAZg6mdTlgEMju0lBsyTaX2BLXSWadWS5XfjiLGh"] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] + diff --git a/regression-test/data/nereids_p0/cte/test_cte_filter_pushdown.out b/regression-test/data/nereids_p0/cte/test_cte_filter_pushdown.out index 0c632f4fc29d01..1988eab0d7199b 100644 --- a/regression-test/data/nereids_p0/cte/test_cte_filter_pushdown.out +++ b/regression-test/data/nereids_p0/cte/test_cte_filter_pushdown.out @@ -10,7 +10,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalResultSink ----PhysicalDistribute ------PhysicalProject ---------hashJoin[INNER_JOIN](m1.k1 = m2.k1) +--------hashJoin[INNER_JOIN] hashCondition=((m1.k1 = m2.k1))otherCondition=() ----------PhysicalDistribute ------------filter((temp.k1 = 1)) --------------PhysicalCteConsumer ( cteId=CTEId#0 ) @@ -32,7 +32,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalResultSink ----PhysicalDistribute ------PhysicalProject ---------hashJoin[INNER_JOIN](m1.k1 = m2.k1) +--------hashJoin[INNER_JOIN] hashCondition=((m1.k1 = m2.k1))otherCondition=() ----------PhysicalDistribute ------------filter((temp.k1 = 1)) --------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.1.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.1.out index 19bef3de9ecbfa..2862a626ce6612 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.1.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.1.out @@ -5,9 +5,9 @@ PhysicalResultSink ----PhysicalDistribute ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) +----------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() ------------PhysicalProject ---------------filter((lineorder.lo_discount <= 3)(lineorder.lo_discount >= 1)(lineorder.lo_quantity < 25)) +--------------filter((lineorder.lo_discount <= 3) and (lineorder.lo_discount >= 1) and (lineorder.lo_quantity < 25)) ----------------PhysicalOlapScan[lineorder] ------------PhysicalDistribute --------------PhysicalProject diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.2.out index f1ac2986577e7c..051e308ad3b4e0 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.2.out @@ -5,9 +5,9 @@ PhysicalResultSink ----PhysicalDistribute ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) +----------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() ------------PhysicalProject ---------------filter((lineorder.lo_quantity <= 35)(lineorder.lo_discount <= 6)(lineorder.lo_discount >= 4)(lineorder.lo_quantity >= 26)) +--------------filter((lineorder.lo_discount <= 6) and (lineorder.lo_discount >= 4) and (lineorder.lo_quantity <= 35) and (lineorder.lo_quantity >= 26)) ----------------PhysicalOlapScan[lineorder] ------------PhysicalDistribute --------------PhysicalProject diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.3.out index 1e4a6a40b51c50..efe26b1ccc9843 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q1.3.out @@ -5,12 +5,12 @@ PhysicalResultSink ----PhysicalDistribute ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) +----------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() ------------PhysicalProject ---------------filter((lineorder.lo_discount <= 7)(lineorder.lo_discount >= 5)(lineorder.lo_quantity <= 35)(lineorder.lo_quantity >= 26)) +--------------filter((lineorder.lo_discount <= 7) and (lineorder.lo_discount >= 5) and (lineorder.lo_quantity <= 35) and (lineorder.lo_quantity >= 26)) ----------------PhysicalOlapScan[lineorder] ------------PhysicalDistribute --------------PhysicalProject -----------------filter((dates.d_year = 1994)(dates.d_weeknuminyear = 6)) +----------------filter((dates.d_weeknuminyear = 6) and (dates.d_year = 1994)) ------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out index 44e6976230157e..0f686b1356b440 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out @@ -8,9 +8,9 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) -------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) ---------------------hashJoin[INNER_JOIN](lineorder.lo_partkey = part.p_partkey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() +--------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_partkey = part.p_partkey))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[lineorder] ----------------------PhysicalDistribute diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out index 7f5570465b93a2..22c54a6645f2a1 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out @@ -8,17 +8,17 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](lineorder.lo_partkey = part.p_partkey) +--------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_partkey = part.p_partkey))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[lineorder] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((part.p_brand >= 'MFGR#2221')(part.p_brand <= 'MFGR#2228')) +--------------------------------filter((part.p_brand <= 'MFGR#2228') and (part.p_brand >= 'MFGR#2221')) ----------------------------------PhysicalOlapScan[part] ------------------------PhysicalDistribute --------------------------PhysicalProject diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out index 10955aaf2295d3..10cb72fe32bad0 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out @@ -8,12 +8,12 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](lineorder.lo_partkey = part.p_partkey) +--------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_partkey = part.p_partkey))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[lineorder] ----------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.1.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.1.out index 2ed9372602d068..982b99bdb341cd 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.1.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.1.out @@ -8,11 +8,11 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](lineorder.lo_custkey = customer.c_custkey) +--------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_custkey = customer.c_custkey))otherCondition=() ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) +------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[lineorder] --------------------------PhysicalDistribute @@ -25,6 +25,6 @@ PhysicalResultSink ----------------------------PhysicalOlapScan[customer] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((dates.d_year <= 1997)(dates.d_year >= 1992)) +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) ------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out index b181c7be1c8b33..15c25139c764ac 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out @@ -8,11 +8,11 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](lineorder.lo_custkey = customer.c_custkey) +--------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_custkey = customer.c_custkey))otherCondition=() ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) +------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[lineorder] --------------------------PhysicalDistribute @@ -25,6 +25,6 @@ PhysicalResultSink ----------------------------PhysicalOlapScan[customer] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((dates.d_year <= 1997)(dates.d_year >= 1992)) +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) ------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out index d7bdbaa77bac02..cca14c087d5365 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out @@ -8,24 +8,24 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](lineorder.lo_custkey = customer.c_custkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_custkey = customer.c_custkey))otherCondition=() ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) +--------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[lineorder] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(s_city IN ('UNITED KI5', 'UNITED KI1')) +--------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) ----------------------------------PhysicalOlapScan[supplier] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter(c_city IN ('UNITED KI5', 'UNITED KI1')) +----------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ------------------------------PhysicalOlapScan[customer] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((dates.d_year <= 1997)(dates.d_year >= 1992)) +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) ------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out index ad9c6d4649631d..79f7f88e89a103 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out @@ -8,16 +8,16 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_custkey = customer.c_custkey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_custkey = customer.c_custkey))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) -------------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[lineorder] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(s_city IN ('UNITED KI5', 'UNITED KI1')) +------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) --------------------------------PhysicalOlapScan[supplier] ------------------------PhysicalDistribute --------------------------PhysicalProject @@ -25,6 +25,6 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[dates] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter(c_city IN ('UNITED KI5', 'UNITED KI1')) +----------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.1.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.1.out index b6af716127deb1..07111ec2da35a9 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.1.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.1.out @@ -8,13 +8,13 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) -------------------hashJoin[INNER_JOIN](lineorder.lo_partkey = part.p_partkey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_partkey = part.p_partkey))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](lineorder.lo_custkey = customer.c_custkey) +------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_custkey = customer.c_custkey))otherCondition=() --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) +----------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[lineorder] ------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out index 8a50b45af6bee3..d4e8df36a8262f 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out @@ -8,14 +8,14 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_partkey = part.p_partkey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_partkey = part.p_partkey))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](lineorder.lo_custkey = customer.c_custkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_custkey = customer.c_custkey))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) -------------------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) +----------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[lineorder] --------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out index 8c2b1194ef9349..6a6660283dc1db 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out @@ -8,14 +8,14 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineorder.lo_custkey = customer.c_custkey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_custkey = customer.c_custkey))otherCondition=() ------------------PhysicalProject --------------------PhysicalOlapScan[customer] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](lineorder.lo_orderdate = dates.d_datekey) -------------------------hashJoin[INNER_JOIN](lineorder.lo_partkey = part.p_partkey) ---------------------------hashJoin[INNER_JOIN](lineorder.lo_suppkey = supplier.s_suppkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_orderdate = dates.d_datekey))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_partkey = part.p_partkey))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[lineorder] ----------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out index ff3e90c1248d7e..9d78edab8fce02 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out @@ -6,7 +6,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------hashAgg[LOCAL] ----------PhysicalProject -------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) +------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk))otherCondition=() --------------PhysicalProject ----------------PhysicalOlapScan[store_returns] --------------PhysicalDistribute @@ -18,14 +18,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) +------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk))otherCondition=() --------------PhysicalDistribute ----------------PhysicalProject ------------------PhysicalOlapScan[customer] --------------PhysicalDistribute -----------------hashJoin[INNER_JOIN](ctr1.ctr_store_sk = ctr2.ctr_store_sk)(cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE)) +----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_store_sk = ctr2.ctr_store_sk))otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store.s_store_sk = ctr1.ctr_store_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = ctr1.ctr_store_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out index f9fa345be63011..fcb2ade8dace09 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out @@ -9,49 +9,49 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) +------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_moy <= 4)(date_dim.d_year = 2001)) +------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------filter(($c$1 OR $c$2)) -------------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) ---------------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk))otherCondition=() +--------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk))otherCondition=() ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk))otherCondition=() ----------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter(ca_county IN ('Storey County', 'Marquette County', 'Warren County', 'Cochran County', 'Kandiyohi County')) +--------------------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) ----------------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[web_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2001)(date_dim.d_moy <= 4)) +--------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[catalog_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_moy <= 4)(date_dim.d_year = 2001)) +------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) --------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out index cd7071884b2f72..3cb337bdcdb3ac 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out @@ -8,9 +8,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk))otherCondition=() ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute @@ -25,9 +25,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk))otherCondition=() ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[web_sales] ----------------------PhysicalDistribute @@ -42,23 +42,23 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(if((year_total > 0.00), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), 0) > if((year_total > 0.00), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), 0)) ---------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) -----------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) +------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id))otherCondition=((if((year_total > 0.00), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), 0) > if((year_total > 0.00), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), 0))) +--------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id))otherCondition=() +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00)) +----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.dyear = 2001)) +----------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002)) +--------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((t_w_secyear.dyear = 2002)(t_w_secyear.sale_type = 'w')) +------------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out index a48c3cceb8f30a..08c4fb6fff050a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out @@ -12,17 +12,17 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk))otherCondition=() --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[web_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= 1998-05-06)(date_dim.d_date >= 1998-04-06)) +----------------------------------filter((date_dim.d_date <= 1998-05-06) and (date_dim.d_date >= 1998-04-06)) ------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(i_category IN ('Books', 'Sports', 'Men')) +------------------------------filter(i_category IN ('Books', 'Men', 'Sports')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out index 441b39ce61bbf2..79a3e5613c8e14 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out @@ -5,34 +5,34 @@ PhysicalResultSink ----PhysicalDistribute ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------PhysicalProject --------------filter((date_dim.d_year = 2001)) ----------------PhysicalOlapScan[date_dim] ------------PhysicalDistribute --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk))otherCondition=() ------------------PhysicalProject --------------------PhysicalOlapScan[store] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)((((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)) OR ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) AND (household_demographics.hd_dep_count = 1))) OR ((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))) AND (household_demographics.hd_dep_count = 1))) +----------------------hashJoin[INNER_JOIN] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk))otherCondition=(((((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)) OR ((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) AND (household_demographics.hd_dep_count = 1))) OR ((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))) AND (household_demographics.hd_dep_count = 1)))) ------------------------PhysicalProject --------------------------filter(((((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'College'))) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '4 yr Degree')))) ----------------------------PhysicalOlapScan[customer_demographics] ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_dep_count = 1))) --------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('SD', 'KS', 'MI') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('MO', 'ND', 'CO') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk))otherCondition=((((ca_state IN ('KS', 'MI', 'SD') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('CO', 'MO', 'ND') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))))) --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter(((((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) +------------------------------------filter(((((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00))) and ((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) --------------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_country = 'United States')((ca_state IN ('SD', 'KS', 'MI') OR ca_state IN ('MO', 'ND', 'CO')) OR ca_state IN ('NH', 'OH', 'TX'))) +------------------------------------filter(((ca_state IN ('KS', 'MI', 'SD') OR ca_state IN ('CO', 'MO', 'ND')) OR ca_state IN ('NH', 'OH', 'TX')) and (customer_address.ca_country = 'United States')) --------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out index 1822d8d547f9bf..490557556dc0f8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN](item.i_brand_id = t.brand_id)(item.i_class_id = t.class_id)(item.i_category_id = t.category_id) +------hashJoin[INNER_JOIN] hashCondition=((item.i_brand_id = t.brand_id) and (item.i_category_id = t.category_id) and (item.i_class_id = t.class_id))otherCondition=() --------PhysicalDistribute ----------PhysicalProject ------------PhysicalOlapScan[item] @@ -13,13 +13,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = iss.i_item_sk) -----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = iss.i_item_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk))otherCondition=() ------------------------PhysicalProject --------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((d1.d_year <= 2002)(d1.d_year >= 2000)) +----------------------------filter((d1.d_year <= 2002) and (d1.d_year >= 2000)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject @@ -29,14 +29,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = ics.i_item_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = ics.i_item_sk))otherCondition=() ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d2.d_date_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d2.d_date_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[catalog_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((d2.d_year >= 2000)(d2.d_year <= 2002)) +------------------------------filter((d2.d_year <= 2002) and (d2.d_year >= 2000)) --------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject @@ -46,14 +46,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = iws.i_item_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = iws.i_item_sk))otherCondition=() ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = d3.d_date_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = d3.d_date_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[web_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((d3.d_year <= 2002)(d3.d_year >= 2000)) +------------------------------filter((d3.d_year <= 2002) and (d3.d_year >= 2000)) --------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject @@ -66,27 +66,27 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------PhysicalUnion --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute -----------------------filter((date_dim.d_year <= 2002)(date_dim.d_year >= 2000)) +----------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ------------------------PhysicalOlapScan[date_dim] --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[catalog_sales] --------------------PhysicalDistribute -----------------------filter((date_dim.d_year <= 2002)(date_dim.d_year >= 2000)) +----------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ------------------------PhysicalOlapScan[date_dim] --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[web_sales] --------------------PhysicalDistribute -----------------------filter((date_dim.d_year >= 2000)(date_dim.d_year <= 2002)) +----------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) ------------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink ------PhysicalTopN @@ -105,15 +105,15 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[RIGHT_SEMI_JOIN](store_sales.ss_item_sk = cross_items.ss_item_sk) +--------------------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk))otherCondition=() ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ------------------------------------------PhysicalDistribute ---------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------------PhysicalOlapScan[store_sales] ----------------------------------------------PhysicalDistribute -------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalOlapScan[item] @@ -128,15 +128,15 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = cross_items.ss_item_sk) +--------------------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk))otherCondition=() ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() ------------------------------------------PhysicalDistribute ---------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------------------PhysicalDistribute -------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalOlapScan[item] @@ -151,15 +151,15 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------PhysicalDistribute ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = cross_items.ss_item_sk) +--------------------------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk))otherCondition=() ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk))otherCondition=() ------------------------------------------PhysicalDistribute ---------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------------PhysicalOlapScan[web_sales] ----------------------------------------------PhysicalDistribute -------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) --------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out index abb2fe6156f5ae..8b0f3345ed9d38 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out @@ -8,18 +8,18 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR ca_state IN ('CA', 'WA', 'GA')) OR (catalog_sales.cs_sales_price > 500.00)) +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk))otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 1)(date_dim.d_year = 2001)) +--------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) ----------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out index 1a6709b3b54619..7609d52b1b5cfa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out @@ -9,18 +9,18 @@ PhysicalResultSink ------------hashAgg[GLOBAL] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](cs1.cs_order_number = cs2.cs_order_number)( not (cs_warehouse_sk = cs_warehouse_sk)) +------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((cs1.cs_order_number = cs2.cs_order_number))otherCondition=(( not (cs_warehouse_sk = cs_warehouse_sk))) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_sales] ---------------------hashJoin[INNER_JOIN](cs1.cs_call_center_sk = call_center.cc_call_center_sk) -----------------------hashJoin[RIGHT_ANTI_JOIN](cs1.cs_order_number = cr1.cr_order_number) +--------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_call_center_sk = call_center.cc_call_center_sk))otherCondition=() +----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((cs1.cs_order_number = cr1.cr_order_number))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------PhysicalOlapScan[catalog_returns] ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](cs1.cs_ship_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](cs1.cs_ship_addr_sk = customer_address.ca_address_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_date_sk = date_dim.d_date_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((cs1.cs_ship_addr_sk = customer_address.ca_address_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute @@ -29,10 +29,10 @@ PhysicalResultSink ------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= 2002-05-31)(date_dim.d_date >= 2002-04-01)) +--------------------------------filter((date_dim.d_date <= 2002-05-31) and (date_dim.d_date >= 2002-04-01)) ----------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(cc_county IN ('Ziebach County', 'Luce County', 'Richland County', 'Daviess County', 'Barrow County')) +--------------------------filter(cc_county IN ('Barrow County', 'Daviess County', 'Luce County', 'Richland County', 'Ziebach County')) ----------------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out index b238c38de00eec..54d30213060125 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out @@ -9,24 +9,24 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk))otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------PhysicalOlapScan[catalog_sales] ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk))otherCondition=() ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk))otherCondition=() --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk))otherCondition=() --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number))otherCondition=() ------------------------------------------PhysicalProject --------------------------------------------PhysicalOlapScan[store_returns] -------------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[store_sales] --------------------------------------------PhysicalDistribute 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 b136f2c263aaf1..980b95e1901ab4 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,30 +10,30 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalRepeat ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](customer.c_current_cdemo_sk = cd2.cd_demo_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk))otherCondition=() ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[customer_demographics] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk))otherCondition=() ----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk) +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk))otherCondition=() ------------------------------------------------PhysicalProject --------------------------------------------------PhysicalOlapScan[catalog_sales] ------------------------------------------------PhysicalDistribute --------------------------------------------------PhysicalProject -----------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree')(cd1.cd_gender = 'F')) +----------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) ------------------------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------------------PhysicalDistribute ----------------------------------------------PhysicalProject @@ -41,10 +41,10 @@ PhysicalResultSink --------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter(c_birth_month IN (1, 2, 4, 7, 8, 10)) +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) ----------------------------------------------PhysicalOlapScan[customer] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter(ca_state IN ('WA', 'GA', 'NC', 'ME', 'WY', 'OK', 'IN')) +--------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) ----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out index 5164ce7840a79b..1dc78a027b4cae 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out @@ -8,21 +8,21 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk)( not (substring(ca_zip, 1, 5) = substring(s_zip, 1, 5))) +----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=(( not (substring(ca_zip, 1, 5) = substring(s_zip, 1, 5)))) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk))otherCondition=() ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[customer] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] ------------------------------------PhysicalDistribute @@ -31,7 +31,7 @@ PhysicalResultSink ------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ----------------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out index dd224940107ef1..fb43ce4addc834 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out @@ -6,7 +6,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------PhysicalDistribute --------hashAgg[LOCAL] ----------PhysicalProject -------------hashJoin[INNER_JOIN](date_dim.d_date_sk = wscs.sold_date_sk) +------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = wscs.sold_date_sk))otherCondition=() --------------PhysicalUnion ----------------PhysicalDistribute ------------------PhysicalProject @@ -22,10 +22,10 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------PhysicalDistribute --------PhysicalQuickSort ----------PhysicalProject -------------hashJoin[INNER_JOIN](expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53)) +------------hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53)))otherCondition=() --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq1) +------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = d_week_seq1))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) @@ -35,7 +35,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --------------------------PhysicalOlapScan[date_dim] --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq2) +------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = d_week_seq2))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out index 301615de702911..9f7938f325b94f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out @@ -12,17 +12,17 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date >= 2002-01-26)(date_dim.d_date <= 2002-02-25)) +----------------------------------filter((date_dim.d_date <= 2002-02-25) and (date_dim.d_date >= 2002-01-26)) ------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(i_category IN ('Shoes', 'Books', 'Women')) +------------------------------filter(i_category IN ('Books', 'Shoes', 'Women')) --------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out index 58c63f10f1101b..c94c8bc0fc7e87 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out @@ -4,22 +4,22 @@ PhysicalResultSink --PhysicalTopN ----PhysicalDistribute ------PhysicalTopN ---------filter((if((inv_before > 0), (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)), NULL) <= 1.5)(if((inv_before > 0), (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)), NULL) >= cast((2.000000 / 3.0) as DOUBLE))) +--------filter((if((inv_before > 0), (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)), NULL) <= 1.5) and (if((inv_before > 0), (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)), NULL) >= cast((2.000000 / 3.0) as DOUBLE))) ----------hashAgg[GLOBAL] ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](inventory.inv_warehouse_sk = warehouse.w_warehouse_sk) ---------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) -----------------------hashJoin[INNER_JOIN](item.i_item_sk = inventory.inv_item_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk))otherCondition=() +--------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = inventory.inv_item_sk))otherCondition=() ------------------------PhysicalOlapScan[inventory] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((item.i_current_price <= 1.49)(item.i_current_price >= 0.99)) +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ------------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((date_dim.d_date >= 2002-01-28)(date_dim.d_date <= 2002-03-29)) +--------------------------filter((date_dim.d_date <= 2002-03-29) and (date_dim.d_date >= 2002-01-28)) ----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out index ec560939d191a6..1a38000011a62d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out @@ -10,8 +10,8 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalRepeat ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) -----------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_item_sk = item.i_item_sk))otherCondition=() ------------------------PhysicalProject --------------------------PhysicalOlapScan[inventory] ------------------------PhysicalDistribute @@ -19,6 +19,6 @@ PhysicalResultSink ----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((date_dim.d_month_seq <= 1199)(date_dim.d_month_seq >= 1188)) +--------------------------filter((date_dim.d_month_seq <= 1199) and (date_dim.d_month_seq >= 1188)) ----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out index f3c41978dc8c93..23c76188a03002 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out @@ -8,8 +8,8 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute @@ -25,7 +25,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------NestedLoopJoin[INNER_JOIN](cast(ssales as DOUBLE) > cast((0.9500 * tpcds_cmax) as DOUBLE)) ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +--------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk))otherCondition=() ----------------PhysicalDistribute ------------------PhysicalProject --------------------PhysicalOlapScan[store_sales] @@ -40,9 +40,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk))otherCondition=() ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute @@ -60,33 +60,33 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------hashAgg[LOCAL] ----------------PhysicalUnion ------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = frequent_ss_items.item_sk) +--------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((catalog_sales.cs_item_sk = frequent_ss_items.item_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalDistribute -------------------------hashJoin[LEFT_SEMI_JOIN](catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk) +------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk))otherCondition=() --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute ---------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject ------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = frequent_ss_items.item_sk) +--------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((web_sales.ws_item_sk = frequent_ss_items.item_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalDistribute -------------------------hashJoin[LEFT_SEMI_JOIN](web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk) +------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk))otherCondition=() --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalOlapScan[web_sales] ------------------------------PhysicalDistribute ---------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) +--------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 2000)) ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out index 0a0601bc0f68ac..7e73cccbde8d25 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out @@ -7,15 +7,15 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) +--------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number))otherCondition=() ----------------PhysicalDistribute ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------hashJoin[INNER_JOIN](store.s_zip = customer_address.ca_zip)(customer.c_current_addr_sk = customer_address.ca_address_sk)( not (c_birth_country = upper(ca_country))) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk) and (store.s_zip = customer_address.ca_zip))otherCondition=(( not (c_birth_country = upper(ca_country)))) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk))otherCondition=() ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out index 81ba434db88870..93bc1d64fafa69 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out @@ -8,33 +8,33 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[catalog_sales] ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk))otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk))otherCondition=() ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk))otherCondition=() ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() ------------------------------------------PhysicalProject --------------------------------------------PhysicalOlapScan[store_sales] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------filter((d1.d_year = 2000)(d1.d_moy = 4)) +----------------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 2000)) ------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((d2.d_moy <= 10)(d2.d_moy >= 4)(d2.d_year = 2000)) +----------------------------------------filter((d2.d_moy <= 10) and (d2.d_moy >= 4) and (d2.d_year = 2000)) ------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject @@ -44,6 +44,6 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[store] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((d3.d_year = 2000)(d3.d_moy <= 10)(d3.d_moy >= 4)) +----------------------filter((d3.d_moy <= 10) and (d3.d_moy >= 4) and (d3.d_year = 2000)) ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out index 04a98730cf6633..fc26df2456563c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out @@ -8,17 +8,17 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk))otherCondition=() ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((customer_demographics.cd_gender = 'M')(customer_demographics.cd_marital_status = 'S')(customer_demographics.cd_education_status = 'Unknown')) +--------------------------------filter((customer_demographics.cd_education_status = 'Unknown') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'S')) ----------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out index 00cb9a518855d6..4ef76ce80daad1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out @@ -11,17 +11,17 @@ PhysicalResultSink ----------------PhysicalProject ------------------PhysicalRepeat --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((customer_demographics.cd_education_status = 'Secondary')(customer_demographics.cd_marital_status = 'D')(customer_demographics.cd_gender = 'F')) +--------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ----------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject @@ -32,6 +32,6 @@ PhysicalResultSink ------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter(s_state IN ('MO', 'AL', 'MI', 'TN', 'LA', 'SC')) +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out index d38a5269ff5fc5..e585932d6f86bf 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out @@ -18,7 +18,7 @@ PhysicalResultSink ------------------------------PhysicalDistribute --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------filter((store_sales.ss_quantity <= 5)((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00)))(store_sales.ss_quantity >= 0)) +------------------------------------filter(((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00))) and (store_sales.ss_quantity <= 5) and (store_sales.ss_quantity >= 0)) --------------------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalLimit @@ -26,7 +26,7 @@ PhysicalResultSink --------------------------------PhysicalDistribute ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00)))(store_sales.ss_quantity <= 10)(store_sales.ss_quantity >= 6)) +--------------------------------------filter(((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00))) and (store_sales.ss_quantity <= 10) and (store_sales.ss_quantity >= 6)) ----------------------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalLimit @@ -34,7 +34,7 @@ PhysicalResultSink ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 1.6E+2)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7.6E+3))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00)))(store_sales.ss_quantity >= 11)(store_sales.ss_quantity <= 15)) +----------------------------------filter(((((store_sales.ss_list_price >= 150.00) AND (store_sales.ss_list_price <= 1.6E+2)) OR ((store_sales.ss_coupon_amt >= 6600.00) AND (store_sales.ss_coupon_amt <= 7.6E+3))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00))) and (store_sales.ss_quantity <= 15) and (store_sales.ss_quantity >= 11)) ------------------------------------PhysicalOlapScan[store_sales] ------------------PhysicalDistribute --------------------PhysicalLimit @@ -42,7 +42,7 @@ PhysicalResultSink ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((store_sales.ss_quantity <= 20)((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00)))(store_sales.ss_quantity >= 16)) +------------------------------filter(((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00))) and (store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 16)) --------------------------------PhysicalOlapScan[store_sales] --------------PhysicalDistribute ----------------PhysicalLimit @@ -50,7 +50,7 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00)))(store_sales.ss_quantity <= 25)(store_sales.ss_quantity >= 21)) +--------------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00))) and (store_sales.ss_quantity <= 25) and (store_sales.ss_quantity >= 21)) ----------------------------PhysicalOlapScan[store_sales] ----------PhysicalDistribute ------------PhysicalLimit @@ -58,6 +58,6 @@ PhysicalResultSink ----------------PhysicalDistribute ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter((store_sales.ss_quantity >= 26)((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00)))(store_sales.ss_quantity <= 30)) +----------------------filter(((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00))) and (store_sales.ss_quantity <= 30) and (store_sales.ss_quantity >= 26)) ------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out index 59a21db91a9f81..39042f54138870 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out @@ -8,32 +8,32 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_sales] ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk))otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk))otherCondition=() ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk))otherCondition=() ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() ------------------------------------------PhysicalProject --------------------------------------------PhysicalOlapScan[store_sales] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------filter((d1.d_year = 1999)(d1.d_moy = 4)) +----------------------------------------------filter((d1.d_moy = 4) and (d1.d_year = 1999)) ------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((d2.d_moy <= 7)(d2.d_moy >= 4)(d2.d_year = 1999)) +----------------------------------------filter((d2.d_moy <= 7) and (d2.d_moy >= 4) and (d2.d_year = 1999)) ------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out index 60ed4b442ce24f..7590a50077fa5d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out @@ -8,9 +8,9 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((dt.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out index 55de401bc4f490..7995bfe61940c8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out @@ -6,10 +6,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------hashAgg[LOCAL] ----------PhysicalProject -------------hashJoin[INNER_JOIN](web_returns.wr_returning_addr_sk = customer_address.ca_address_sk) +------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returning_addr_sk = customer_address.ca_address_sk))otherCondition=() --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[web_returns] --------------------PhysicalDistribute @@ -24,13 +24,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------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)) +------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state))otherCondition=((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) +----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[customer] ----------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out index 83184d7b60981f..72e8d4025efb44 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out @@ -6,15 +6,15 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------hashAgg[LOCAL] ----------PhysicalProject -------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk))otherCondition=() --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter(d_qoy IN (1, 2, 3)(ss.d_year = 2000)) +------------------------filter((ss.d_year = 2000) and d_qoy IN (1, 2, 3)) --------------------------PhysicalOlapScan[date_dim] --------------PhysicalDistribute ----------------PhysicalProject @@ -25,15 +25,15 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +--------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_bill_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------PhysicalDistribute ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[web_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((ws.d_year = 2000)d_qoy IN (1, 2, 3)) +--------------------------filter((ws.d_year = 2000) and d_qoy IN (1, 2, 3)) ----------------------------PhysicalOlapScan[date_dim] ----------------PhysicalDistribute ------------------PhysicalProject @@ -43,34 +43,34 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute ----------PhysicalQuickSort ------------PhysicalProject ---------------hashJoin[INNER_JOIN](ws1.ca_county = ws3.ca_county)(if((web_sales > 0.00), (cast(web_sales as DOUBLE) / cast(web_sales as DOUBLE)), NULL) > if((store_sales > 0.00), (cast(store_sales as DOUBLE) / cast(store_sales as DOUBLE)), NULL)) +--------------hashJoin[INNER_JOIN] hashCondition=((ws1.ca_county = ws3.ca_county))otherCondition=((if((web_sales > 0.00), (cast(web_sales as DOUBLE) / cast(web_sales as DOUBLE)), NULL) > if((store_sales > 0.00), (cast(store_sales as DOUBLE) / cast(store_sales as DOUBLE)), NULL))) ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((ws3.d_year = 2000)(ws3.d_qoy = 3)) +--------------------filter((ws3.d_qoy = 3) and (ws3.d_year = 2000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](ss2.ca_county = ss3.ca_county) +------------------hashJoin[INNER_JOIN] hashCondition=((ss2.ca_county = ss3.ca_county))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((ss3.d_year = 2000)(ss3.d_qoy = 3)) +------------------------filter((ss3.d_qoy = 3) and (ss3.d_year = 2000)) --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------hashJoin[INNER_JOIN](ws1.ca_county = ws2.ca_county)(if((web_sales > 0.00), (cast(web_sales as DOUBLE) / cast(web_sales as DOUBLE)), NULL) > if((store_sales > 0.00), (cast(store_sales as DOUBLE) / cast(store_sales as DOUBLE)), NULL)) -----------------------hashJoin[INNER_JOIN](ss1.ca_county = ws1.ca_county) -------------------------hashJoin[INNER_JOIN](ss1.ca_county = ss2.ca_county) +--------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ca_county = ws2.ca_county))otherCondition=((if((web_sales > 0.00), (cast(web_sales as DOUBLE) / cast(web_sales as DOUBLE)), NULL) > if((store_sales > 0.00), (cast(store_sales as DOUBLE) / cast(store_sales as DOUBLE)), NULL))) +----------------------hashJoin[INNER_JOIN] hashCondition=((ss1.ca_county = ws1.ca_county))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((ss1.ca_county = ss2.ca_county))otherCondition=() --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((ss1.d_year = 2000)(ss1.d_qoy = 1)) +------------------------------filter((ss1.d_qoy = 1) and (ss1.d_year = 2000)) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((ss2.d_year = 2000)(ss2.d_qoy = 2)) +------------------------------filter((ss2.d_qoy = 2) and (ss2.d_year = 2000)) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((ws1.d_year = 2000)(ws1.d_qoy = 1)) +----------------------------filter((ws1.d_qoy = 1) and (ws1.d_year = 2000)) ------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((ws2.d_qoy = 2)(ws2.d_year = 2000)) +--------------------------filter((ws2.d_qoy = 2) and (ws2.d_year = 2000)) ----------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out index 2f2c494bb25224..7e5fa70c52e743 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out @@ -12,8 +12,8 @@ PhysicalResultSink ------------------PhysicalQuickSort --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) ---------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalDistribute @@ -22,6 +22,6 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_date >= 1999-01-07)(date_dim.d_date <= 1999-04-07)) +------------------------------filter((date_dim.d_date <= 1999-04-07) and (date_dim.d_date >= 1999-01-07)) --------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out index 1feccb0a5bee20..0d3bf5f7a640f5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------hashAgg[LOCAL] --------------PhysicalUnion ----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((item.i_manufact_id = item.i_manufact_id))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((item.i_category = 'Home')) @@ -18,13 +18,13 @@ PhysicalResultSink ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk))otherCondition=() +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------------PhysicalOlapScan[store_sales] ------------------------------------PhysicalDistribute ---------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +--------------------------------------filter((date_dim.d_moy = 1) and (date_dim.d_year = 2002)) ----------------------------------------PhysicalOlapScan[date_dim] ----------------------------------PhysicalDistribute ------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) @@ -32,7 +32,7 @@ PhysicalResultSink ------------------------------PhysicalDistribute --------------------------------PhysicalOlapScan[item] ----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((item.i_manufact_id = item.i_manufact_id))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((item.i_category = 'Home')) @@ -41,14 +41,14 @@ PhysicalResultSink ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() ------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------------PhysicalDistribute -------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalOlapScan[catalog_sales] --------------------------------------PhysicalDistribute -----------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +----------------------------------------filter((date_dim.d_moy = 1) and (date_dim.d_year = 2002)) ------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------PhysicalDistribute ------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) @@ -56,7 +56,7 @@ PhysicalResultSink ------------------------------PhysicalDistribute --------------------------------PhysicalOlapScan[item] ----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((item.i_manufact_id = item.i_manufact_id))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((item.i_category = 'Home')) @@ -65,15 +65,15 @@ PhysicalResultSink ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk))otherCondition=() ------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_bill_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------------PhysicalDistribute -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalOlapScan[web_sales] --------------------------------------PhysicalDistribute -----------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +----------------------------------------filter((date_dim.d_moy = 1) and (date_dim.d_year = 2002)) ------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------PhysicalDistribute ------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out index 504194d074f9f8..4ef89bd92c1531 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out @@ -5,30 +5,30 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalQuickSort --------PhysicalProject -----------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) +----------hashJoin[INNER_JOIN] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk))otherCondition=() ------------PhysicalProject --------------PhysicalOlapScan[customer] ------------PhysicalDistribute ---------------filter((dn.cnt <= 20)(dn.cnt >= 15)) +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28)))d_year IN (1998, 1999, 2000)) +----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(hd_buy_potential IN ('1001-5000', '0-500')(household_demographics.hd_vehicle_count > 0)(if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2)) +--------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(s_county IN ('Ziebach County', 'Daviess County', 'Walker County', 'Richland County', 'Barrow County', 'Franklin Parish', 'Williamson County', 'Luce County')) +------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) --------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out index 8b3fcceebe9448..47b9f1061ab02b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out @@ -9,26 +9,26 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) +------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) +------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------filter(($c$1 OR $c$2)) -------------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) ---------------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk))otherCondition=() +--------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk))otherCondition=() ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk))otherCondition=() ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk))otherCondition=() ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject --------------------------------------------PhysicalOlapScan[customer] @@ -40,20 +40,20 @@ PhysicalResultSink --------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[web_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) +--------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[catalog_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) +------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) --------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out index 60a0b05eb09f52..395003d42c0e09 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out @@ -15,11 +15,11 @@ PhysicalResultSink ------------------------hashAgg[LOCAL] --------------------------PhysicalRepeat ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ---------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk))otherCondition=() ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] ----------------------------------------PhysicalDistribute @@ -31,6 +31,6 @@ PhysicalResultSink --------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter(s_state IN ('SD', 'TN', 'GA', 'SC', 'MO', 'AL', 'MI', 'OH')) +------------------------------------filter(s_state IN ('AL', 'GA', 'MI', 'MO', 'OH', 'SC', 'SD', 'TN')) --------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out index f9c9f7ec3ea664..c18e9805b4ad64 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out @@ -8,22 +8,22 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() ------------------PhysicalProject --------------------PhysicalOlapScan[catalog_sales] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) -------------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = inventory.inv_date_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_item_sk = item.i_item_sk))otherCondition=() --------------------------PhysicalProject -----------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) +----------------------------filter((inventory.inv_quantity_on_hand <= 500) and (inventory.inv_quantity_on_hand >= 100)) ------------------------------PhysicalOlapScan[inventory] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 75.00)i_manufact_id IN (707, 747, 856, 1000)(item.i_current_price >= 45.00)) +------------------------------filter((item.i_current_price <= 75.00) and (item.i_current_price >= 45.00) and i_manufact_id IN (1000, 707, 747, 856)) --------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_date >= 1999-02-21)(date_dim.d_date <= 1999-04-22)) +----------------------------filter((date_dim.d_date <= 1999-04-22) and (date_dim.d_date >= 1999-02-21)) ------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out index d8ce696005c514..5f163f63af82a9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out @@ -12,15 +12,15 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk))otherCondition=() --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) +------------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject @@ -29,15 +29,15 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk))otherCondition=() --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[catalog_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) +------------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject @@ -46,15 +46,15 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk))otherCondition=() --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[web_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) +------------------------------------filter((date_dim.d_month_seq <= 1194) and (date_dim.d_month_seq >= 1183)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out index 9a39ef048d3b00..5852f1e29cbb8e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out @@ -8,13 +8,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](inventory.inv_warehouse_sk = warehouse.w_warehouse_sk) -------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) ---------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_item_sk = item.i_item_sk))otherCondition=() +--------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------PhysicalOlapScan[inventory] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(d_moy IN (1, 2)(date_dim.d_year = 1998)) +--------------------------filter((date_dim.d_year = 1998) and d_moy IN (1, 2)) ----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject @@ -27,7 +27,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalQuickSort ----------PhysicalProject -------------hashJoin[INNER_JOIN](inv1.w_warehouse_sk = inv2.w_warehouse_sk)(inv1.i_item_sk = inv2.i_item_sk) +------------hashJoin[INNER_JOIN] hashCondition=((inv1.i_item_sk = inv2.i_item_sk) and (inv1.w_warehouse_sk = inv2.w_warehouse_sk))otherCondition=() --------------PhysicalDistribute ----------------PhysicalProject ------------------filter((inv1.d_moy = 1)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out index a784f31be987c6..63f3bf16425e9a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out @@ -8,9 +8,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[store_sales] @@ -19,16 +19,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalOlapScan[customer] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter(d_year IN (2000, 1999)) +----------------------filter(d_year IN (1999, 2000)) ------------------------PhysicalOlapScan[date_dim] ------PhysicalProject --------hashAgg[GLOBAL] ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer.c_customer_sk = catalog_sales.cs_bill_customer_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[catalog_sales] @@ -37,16 +37,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalOlapScan[customer] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter(d_year IN (2000, 1999)) +----------------------filter(d_year IN (1999, 2000)) ------------------------PhysicalOlapScan[date_dim] ------PhysicalProject --------hashAgg[GLOBAL] ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[web_sales] @@ -55,43 +55,43 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalOlapScan[customer] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter(d_year IN (2000, 1999)) +----------------------filter(d_year IN (1999, 2000)) ------------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(if((year_total > 0.000000), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), NULL) > if((year_total > 0.000000), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), NULL)) +------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id))otherCondition=((if((year_total > 0.000000), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), NULL) > if((year_total > 0.000000), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), NULL))) --------------PhysicalProject -----------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)(if((year_total > 0.000000), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), NULL) > if((year_total > 0.000000), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), NULL)) +--------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id))otherCondition=((if((year_total > 0.000000), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), NULL) > if((year_total > 0.000000), (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)), NULL))) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) ---------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_firstyear.customer_id) +------------------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id))otherCondition=() ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's')) +--------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c')) +--------------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) ----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000)) +------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((t_c_secyear.dyear = 2000)(t_c_secyear.sale_type = 'c')) +--------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((t_w_firstyear.dyear = 1999)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.year_total > 0.000000)) +----------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((t_w_secyear.sale_type = 'w')(t_w_secyear.dyear = 2000)) +------------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out index 6c1818fbfdf8e4..04c0b53f0db75f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out @@ -8,22 +8,22 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) +--------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[catalog_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((item.i_current_price >= 0.99)(item.i_current_price <= 1.49)) +------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) --------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_date >= 2001-03-03)(date_dim.d_date <= 2001-05-02)) +----------------------------filter((date_dim.d_date <= 2001-05-02) and (date_dim.d_date >= 2001-03-03)) ------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out index d60ffc848136fb..88bbf3f20b68a5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out @@ -8,9 +8,9 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](item.i_manufact = i1.i_manufact) +----------------hashJoin[INNER_JOIN] hashCondition=((item.i_manufact = i1.i_manufact))otherCondition=() ------------------PhysicalProject ---------------------filter((i1.i_manufact_id >= 748)(i1.i_manufact_id <= 788)) +--------------------filter((i1.i_manufact_id <= 788) and (i1.i_manufact_id >= 748)) ----------------------PhysicalOlapScan[item] ------------------PhysicalDistribute --------------------PhysicalProject @@ -19,6 +19,6 @@ PhysicalResultSink --------------------------PhysicalDistribute ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter((((((((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('violet', 'chiffon')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('small', 'extra large'))) OR ((((item.i_category = 'Women') AND i_color IN ('tomato', 'blanched')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium'))) OR ((((item.i_category = 'Women') AND i_color IN ('lime', 'almond')) AND i_units IN ('Dram', 'Box')) AND i_size IN ('small', 'extra large'))) OR ((((item.i_category = 'Men') AND i_color IN ('chartreuse', 'blue')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('large', 'N/A'))) OR ((((item.i_category = 'Men') AND i_color IN ('tan', 'dodger')) AND i_units IN ('Tsp', 'Bunch')) AND i_size IN ('economy', 'medium'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Pallet', 'Gram')) AND i_size IN ('large', 'N/A'))) OR ((((item.i_category = 'Men') AND i_color IN ('spring', 'indian')) AND i_units IN ('Unknown', 'Carton')) AND i_size IN ('economy', 'medium')))) +--------------------------------filter((((((((((((item.i_category = 'Women') AND i_color IN ('aquamarine', 'gainsboro')) AND i_units IN ('Dozen', 'Ounce')) AND i_size IN ('economy', 'medium')) OR ((((item.i_category = 'Women') AND i_color IN ('chiffon', 'violet')) AND i_units IN ('Pound', 'Ton')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Women') AND i_color IN ('blanched', 'tomato')) AND i_units IN ('Case', 'Tbl')) AND i_size IN ('economy', 'medium'))) OR ((((item.i_category = 'Women') AND i_color IN ('almond', 'lime')) AND i_units IN ('Box', 'Dram')) AND i_size IN ('extra large', 'small'))) OR ((((item.i_category = 'Men') AND i_color IN ('blue', 'chartreuse')) AND i_units IN ('Each', 'Oz')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('dodger', 'tan')) AND i_units IN ('Bunch', 'Tsp')) AND i_size IN ('economy', 'medium'))) OR ((((item.i_category = 'Men') AND i_color IN ('peru', 'saddle')) AND i_units IN ('Gram', 'Pallet')) AND i_size IN ('N/A', 'large'))) OR ((((item.i_category = 'Men') AND i_color IN ('indian', 'spring')) AND i_units IN ('Carton', 'Unknown')) AND i_size IN ('economy', 'medium')))) ----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out index 298efbd86325e4..84b53306efabb3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out @@ -8,8 +8,8 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((dt.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute @@ -18,6 +18,6 @@ PhysicalResultSink --------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((dt.d_year = 2002)(dt.d_moy = 11)) +----------------------filter((dt.d_moy = 11) and (dt.d_year = 2002)) ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out index 53978bf7a7e916..894dc43670db26 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out @@ -8,8 +8,8 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out index 8028e752fd4acc..9c3796dc039633 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out @@ -5,9 +5,9 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalTopN --------PhysicalProject -----------hashJoin[INNER_JOIN](asceding.rnk = descending.rnk) +----------hashJoin[INNER_JOIN] hashCondition=((asceding.rnk = descending.rnk))otherCondition=() ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](i1.i_item_sk = asceding.item_sk) +--------------hashJoin[INNER_JOIN] hashCondition=((i1.i_item_sk = asceding.item_sk))otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[item] ----------------PhysicalDistribute @@ -34,10 +34,10 @@ PhysicalResultSink ----------------------------------------------PhysicalDistribute ------------------------------------------------hashAgg[LOCAL] --------------------------------------------------PhysicalProject -----------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) +----------------------------------------------------filter((store_sales.ss_store_sk = 146) and ss_addr_sk IS NULL) ------------------------------------------------------PhysicalOlapScan[store_sales] ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](i2.i_item_sk = descending.item_sk) +--------------hashJoin[INNER_JOIN] hashCondition=((i2.i_item_sk = descending.item_sk))otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[item] ----------------PhysicalDistribute @@ -64,6 +64,6 @@ PhysicalResultSink ----------------------------------------------PhysicalDistribute ------------------------------------------------hashAgg[LOCAL] --------------------------------------------------PhysicalProject -----------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) +----------------------------------------------------filter((store_sales.ss_store_sk = 146) and ss_addr_sk IS NULL) ------------------------------------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out index 83d87e1582a246..8bb11d71e2aab3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out @@ -8,27 +8,27 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR $c$1)) -------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((item.i_item_id = item.i_item_id))otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk))otherCondition=() ------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[web_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_qoy = 2)(date_dim.d_year = 2000)) +--------------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) ----------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk))otherCondition=() ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[customer] @@ -37,6 +37,6 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[customer_address] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter(i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)) +------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) --------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out index a21333fa4286af..60542cf4c2e083 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out @@ -5,22 +5,22 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalTopN --------PhysicalProject -----------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk)( not (ca_city = bought_city)) +----------hashJoin[INNER_JOIN] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk))otherCondition=(( not (ca_city = bought_city))) ------------PhysicalDistribute --------------PhysicalProject ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter(d_dow IN (0, 6)d_year IN (1999, 2000, 2001)) +------------------------------------filter(d_dow IN (0, 6) and d_year IN (1999, 2000, 2001)) --------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject @@ -28,13 +28,13 @@ PhysicalResultSink ------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(s_city IN ('Five Points', 'Centerville', 'Oak Grove', 'Fairview', 'Liberty')) +--------------------------------filter(s_city IN ('Centerville', 'Fairview', 'Five Points', 'Liberty', 'Oak Grove')) ----------------------------------PhysicalOlapScan[store] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[customer_address] ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk) +--------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk))otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] ----------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out index 12b9c7211f6e1f..d54603a48c0f84 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out @@ -13,10 +13,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute @@ -35,14 +35,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute ----------PhysicalTopN ------------PhysicalProject ---------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_store_name = v1_lag.s_store_name)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) -----------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_store_name = v1_lead.s_store_name)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) +--------------hashJoin[INNER_JOIN] hashCondition=((v1.i_brand = v1_lag.i_brand) and (v1.i_category = v1_lag.i_category) and (v1.rn = expr_(rn + 1)) and (v1.s_company_name = v1_lag.s_company_name) and (v1.s_store_name = v1_lag.s_store_name))otherCondition=() +----------------hashJoin[INNER_JOIN] hashCondition=((v1.i_brand = v1_lead.i_brand) and (v1.i_category = v1_lead.i_category) and (v1.rn = expr_(rn - 1)) and (v1.s_company_name = v1_lead.s_company_name) and (v1.s_store_name = v1_lead.s_store_name))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((if((avg_monthly_sales > 0.0000), (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)), NULL) > 0.1)(v2.d_year = 2001)(v2.avg_monthly_sales > 0.0000)) +----------------------filter((if((avg_monthly_sales > 0.0000), (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)), NULL) > 0.1) and (v2.avg_monthly_sales > 0.0000) and (v2.d_year = 2001)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out index 9d1b3575094e64..c26a22c632a9c8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out @@ -5,21 +5,21 @@ PhysicalResultSink ----PhysicalDistribute ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +----------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk))otherCondition=() ------------PhysicalDistribute --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)(((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) +----------------------hashJoin[INNER_JOIN] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk))otherCondition=((((((customer_demographics.cd_marital_status = 'U') AND (customer_demographics.cd_education_status = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('MD', 'MN', 'IA') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('VA', 'IL', 'TX') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('MI', 'WI', 'IN') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk))otherCondition=((((ca_state IN ('IA', 'MD', 'MN') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('IL', 'TX', 'VA') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('IN', 'MI', 'WI') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) ----------------------------PhysicalProject -------------------------------filter(((((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) +------------------------------filter(((((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))) and ((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) --------------------------------PhysicalOlapScan[store_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(((ca_state IN ('MD', 'MN', 'IA') OR ca_state IN ('VA', 'IL', 'TX')) OR ca_state IN ('MI', 'WI', 'IN'))(customer_address.ca_country = 'United States')) +--------------------------------filter(((ca_state IN ('IA', 'MD', 'MN') OR ca_state IN ('IL', 'TX', 'VA')) OR ca_state IN ('IN', 'MI', 'WI')) and (customer_address.ca_country = 'United States')) ----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalDistribute --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out index b5dc9c3f7050a1..80fae98da42166 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out @@ -20,17 +20,17 @@ PhysicalResultSink ----------------------------------PhysicalDistribute ------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](ws.ws_order_number = wr.wr_order_number)(item = wr.wr_item_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((item = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number))otherCondition=() ------------------------------------------PhysicalProject --------------------------------------------filter((wr.wr_return_amt > 10000.00)) ----------------------------------------------PhysicalOlapScan[web_returns] -------------------------------------------hashJoin[INNER_JOIN](ws.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------filter((ws.ws_net_paid > 0.00)(ws.ws_quantity > 0)(ws.ws_net_profit > 1.00)) +----------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) ------------------------------------------------PhysicalOlapScan[web_sales] --------------------------------------------PhysicalDistribute ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) --------------------------------------------------PhysicalOlapScan[date_dim] ------------PhysicalDistribute --------------PhysicalProject @@ -46,17 +46,17 @@ PhysicalResultSink ----------------------------------PhysicalDistribute ------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](cs.cs_order_number = cr.cr_order_number)(item = cr.cr_item_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs.cs_order_number = cr.cr_order_number) and (item = cr.cr_item_sk))otherCondition=() ------------------------------------------PhysicalProject --------------------------------------------filter((cr.cr_return_amount > 10000.00)) ----------------------------------------------PhysicalOlapScan[catalog_returns] -------------------------------------------hashJoin[INNER_JOIN](cs.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------filter((cs.cs_net_paid > 0.00)(cs.cs_quantity > 0)(cs.cs_net_profit > 1.00)) +----------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) ------------------------------------------------PhysicalOlapScan[catalog_sales] --------------------------------------------PhysicalDistribute ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) --------------------------------------------------PhysicalOlapScan[date_dim] ------------PhysicalDistribute --------------PhysicalProject @@ -72,16 +72,16 @@ PhysicalResultSink ----------------------------------PhysicalDistribute ------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](sts.ss_ticket_number = sr.sr_ticket_number)(item = sr.sr_item_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((item = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number))otherCondition=() ------------------------------------------PhysicalProject --------------------------------------------filter((sr.sr_return_amt > 10000.00)) ----------------------------------------------PhysicalOlapScan[store_returns] -------------------------------------------hashJoin[INNER_JOIN](sts.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------filter((sts.ss_quantity > 0)(sts.ss_net_profit > 1.00)(sts.ss_net_paid > 0.00)) +----------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) ------------------------------------------------PhysicalOlapScan[store_sales] --------------------------------------------PhysicalDistribute ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) --------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out index 9eeca7c6570bbe..7ef6edcb689f5d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out @@ -15,8 +15,8 @@ PhysicalResultSink ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](salesreturns.store_sk = store.s_store_sk) ---------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((salesreturns.store_sk = store.s_store_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((salesreturns.date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalUnion ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject @@ -25,7 +25,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_returns] ----------------------------------PhysicalDistribute -------------------------------------filter((date_dim.d_date <= 2000-09-02)(date_dim.d_date >= 2000-08-19)) +------------------------------------filter((date_dim.d_date <= 2000-09-02) and (date_dim.d_date >= 2000-08-19)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalOlapScan[store] @@ -34,8 +34,8 @@ PhysicalResultSink ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](salesreturns.page_sk = catalog_page.cp_catalog_page_sk) ---------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((salesreturns.page_sk = catalog_page.cp_catalog_page_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((salesreturns.date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalUnion ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject @@ -44,7 +44,7 @@ PhysicalResultSink --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[catalog_returns] ----------------------------------PhysicalDistribute -------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) +------------------------------------filter((date_dim.d_date <= 2000-09-02) and (date_dim.d_date >= 2000-08-19)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalOlapScan[catalog_page] @@ -53,19 +53,19 @@ PhysicalResultSink ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](salesreturns.wsr_web_site_sk = web_site.web_site_sk) ---------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((salesreturns.wsr_web_site_sk = web_site.web_site_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((salesreturns.date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalUnion ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[web_sales] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = web_sales.ws_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = web_sales.ws_item_sk) and (web_returns.wr_order_number = web_sales.ws_order_number))otherCondition=() ------------------------------------------PhysicalOlapScan[web_sales] ------------------------------------------PhysicalOlapScan[web_returns] ----------------------------------PhysicalDistribute -------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) +------------------------------------filter((date_dim.d_date <= 2000-09-02) and (date_dim.d_date >= 2000-08-19)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out index 50219078fe9b58..a21e720046b30a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out @@ -8,23 +8,23 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] -----------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_returns] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((d2.d_year = 2001)(d2.d_moy = 8)) +----------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) ------------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out index b8d6435601a5d9..50eb1c4bf3138f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out @@ -9,7 +9,7 @@ PhysicalResultSink ------------PhysicalQuickSort --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[FULL_OUTER_JOIN](web.item_sk = store.item_sk)(web.d_date = store.d_date) +------------------hashJoin[FULL_OUTER_JOIN] hashCondition=((web.d_date = store.d_date) and (web.item_sk = store.item_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalWindow ------------------------PhysicalQuickSort @@ -19,12 +19,12 @@ PhysicalResultSink --------------------------------PhysicalDistribute ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) +--------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ----------------------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------PhysicalWindow @@ -35,11 +35,11 @@ PhysicalResultSink --------------------------------PhysicalDistribute ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[web_sales] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) +--------------------------------------------filter((date_dim.d_month_seq <= 1227) and (date_dim.d_month_seq >= 1216)) ----------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out index 66ea8b1873475e..053175f95a8cd8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out @@ -8,8 +8,8 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((dt.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute @@ -18,6 +18,6 @@ PhysicalResultSink --------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((dt.d_moy = 12)(dt.d_year = 2002)) +----------------------filter((dt.d_moy = 12) and (dt.d_year = 2002)) ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out index ad58bc9f058bd5..11095f6f345906 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out @@ -14,18 +14,18 @@ PhysicalResultSink ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ------------------------------------------PhysicalProject --------------------------------------------PhysicalOlapScan[store_sales] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) +----------------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) ------------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out index f12974631c6289..dc15ec76069f8b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out @@ -16,17 +16,17 @@ PhysicalResultSink --------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) ----------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](my_customers.c_customer_sk = store_sales.ss_customer_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_customer_sk = store_sales.ss_customer_sk))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN](customer_address.ca_county = store.s_county)(customer_address.ca_state = store.s_state) +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state))otherCondition=() ----------------------------------------------PhysicalDistribute -------------------------------------------------hashJoin[INNER_JOIN](my_customers.c_current_addr_sk = customer_address.ca_address_sk) +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk))otherCondition=() --------------------------------------------------PhysicalProject ----------------------------------------------------PhysicalOlapScan[customer_address] --------------------------------------------------PhysicalDistribute @@ -35,14 +35,14 @@ PhysicalResultSink --------------------------------------------------------PhysicalDistribute ----------------------------------------------------------hashAgg[LOCAL] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN](customer.c_customer_sk = cs_or_ws_sales.customer_sk) +--------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = cs_or_ws_sales.customer_sk))otherCondition=() ----------------------------------------------------------------PhysicalProject ------------------------------------------------------------------PhysicalOlapScan[customer] ----------------------------------------------------------------PhysicalDistribute ------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk) +--------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.item_sk = item.i_item_sk) +------------------------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((cs_or_ws_sales.item_sk = item.i_item_sk))otherCondition=() --------------------------------------------------------------------------PhysicalUnion ----------------------------------------------------------------------------PhysicalDistribute ------------------------------------------------------------------------------PhysicalProject @@ -52,11 +52,11 @@ PhysicalResultSink --------------------------------------------------------------------------------PhysicalOlapScan[web_sales] --------------------------------------------------------------------------PhysicalDistribute ----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------filter((item.i_class = 'maternity')(item.i_category = 'Women')) +------------------------------------------------------------------------------filter((item.i_category = 'Women') and (item.i_class = 'maternity')) --------------------------------------------------------------------------------PhysicalOlapScan[item] ----------------------------------------------------------------------PhysicalDistribute ------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +--------------------------------------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) ----------------------------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------PhysicalDistribute ------------------------------------------------PhysicalProject @@ -71,7 +71,7 @@ PhysicalResultSink --------------------------------------PhysicalDistribute ----------------------------------------hashAgg[LOCAL] ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +--------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) ----------------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalAssertNumRows @@ -80,6 +80,6 @@ PhysicalResultSink ------------------------------------PhysicalDistribute --------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +------------------------------------------filter((date_dim.d_moy = 5) and (date_dim.d_year = 1998)) --------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out index 1215eb4637bf8e..c0dbead7679e94 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out @@ -8,8 +8,8 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute @@ -18,6 +18,6 @@ PhysicalResultSink --------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 2000)) +----------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out index c450ca9175c43f..f18f25836cdab5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out @@ -13,21 +13,21 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute -------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------filter((date_dim.d_moy = 2) and (date_dim.d_year = 2000)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute -----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((item.i_item_id = item.i_item_id))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) +----------------------------------------filter(i_color IN ('cyan', 'green', 'powder')) ------------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalDistribute ------------------------------filter((customer_address.ca_gmt_offset = -6.00)) @@ -37,45 +37,45 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------filter((customer_address.ca_gmt_offset = -6.00)) ------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------PhysicalDistribute -------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------filter((date_dim.d_moy = 2) and (date_dim.d_year = 2000)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute -----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((item.i_item_id = item.i_item_id))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) +----------------------------------------filter(i_color IN ('cyan', 'green', 'powder')) ------------------------------------------PhysicalOlapScan[item] ----------------PhysicalProject ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_bill_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------filter((customer_address.ca_gmt_offset = -6.00)) ------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalOlapScan[web_sales] ----------------------------------PhysicalDistribute -------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------filter((date_dim.d_moy = 2) and (date_dim.d_year = 2000)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute -----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((item.i_item_id = item.i_item_id))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) +----------------------------------------filter(i_color IN ('cyan', 'green', 'powder')) ------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out index 2de32702dc99aa..8458fc4f5513f0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out @@ -13,11 +13,11 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk))otherCondition=() ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() --------------------------------PhysicalDistribute -----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_sales] ------------------------------------PhysicalDistribute @@ -36,14 +36,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute ----------PhysicalTopN ------------PhysicalProject ---------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.cc_name = v1_lag.cc_name)(v1.rn = expr_(rn + 1)) -----------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.cc_name = v1_lead.cc_name)(v1.rn = expr_(rn - 1)) +--------------hashJoin[INNER_JOIN] hashCondition=((v1.cc_name = v1_lag.cc_name) and (v1.i_brand = v1_lag.i_brand) and (v1.i_category = v1_lag.i_category) and (v1.rn = expr_(rn + 1)))otherCondition=() +----------------hashJoin[INNER_JOIN] hashCondition=((v1.cc_name = v1_lead.cc_name) and (v1.i_brand = v1_lead.i_brand) and (v1.i_category = v1_lead.i_category) and (v1.rn = expr_(rn - 1)))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((v2.d_year = 1999)(if((avg_monthly_sales > 0.0000), (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)), NULL) > 0.1)(v2.avg_monthly_sales > 0.0000)) +----------------------filter((if((avg_monthly_sales > 0.0000), (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)), NULL) > 0.1) and (v2.avg_monthly_sales > 0.0000) and (v2.d_year = 1999)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out index cd7b4f2257fedf..36bcd02e220ab3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out @@ -5,16 +5,16 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalTopN --------PhysicalProject -----------hashJoin[INNER_JOIN](ss_items.item_id = cs_items.item_id)(cast(ss_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) +----------hashJoin[INNER_JOIN] hashCondition=((ss_items.item_id = cs_items.item_id))otherCondition=((cast(cs_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE)) and (cast(cs_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) and (cast(cs_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE)) and (cast(cs_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE)) and (cast(ss_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE)) and (cast(ss_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))) ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +----------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date))otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute @@ -25,7 +25,7 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +----------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute @@ -34,16 +34,16 @@ PhysicalResultSink ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_date = 2001-03-24)) ----------------------------------------PhysicalOlapScan[date_dim] -------------hashJoin[INNER_JOIN](ss_items.item_id = ws_items.item_id)(cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) +------------hashJoin[INNER_JOIN] hashCondition=((ss_items.item_id = ws_items.item_id))otherCondition=((cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) and (cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE)) and (cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))) --------------PhysicalProject ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date))otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute @@ -54,7 +54,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute @@ -68,10 +68,10 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date))otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk))otherCondition=() --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[web_sales] @@ -83,7 +83,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute 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 d5d282966a7bf5..0e4dcd07bd3b7f 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 @@ -6,7 +6,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------hashAgg[LOCAL] ----------PhysicalProject -------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() --------------PhysicalProject ----------------PhysicalOlapScan[store_sales] --------------PhysicalDistribute @@ -17,34 +17,34 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](y.s_store_id1 = x.s_store_id2)(expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 52)) +------------hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 52)) and (y.s_store_id1 = x.s_store_id2))otherCondition=() --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((wss.ss_store_sk = store.s_store_sk))otherCondition=() --------------------PhysicalDistribute -----------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1) +----------------------hashJoin[INNER_JOIN] hashCondition=((d.d_week_seq = d_week_seq1))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((d.d_month_seq <= 1207)(d.d_month_seq >= 1196)) +----------------------------filter((d.d_month_seq <= 1207) and (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) +------------------hashJoin[INNER_JOIN] hashCondition=((wss.ss_store_sk = store.s_store_sk))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) +------------------------hashJoin[INNER_JOIN] hashCondition=((d.d_week_seq = d_week_seq2))otherCondition=() --------------------------PhysicalDistribute ----------------------------PhysicalProject ------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((d.d_month_seq <= 1219)(d.d_month_seq >= 1208)) +------------------------------filter((d.d_month_seq <= 1219) and (d.d_month_seq >= 1208)) --------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out index 9603a3c9d0b9df..9d1fd501257eec 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out @@ -9,21 +9,21 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](j.i_category = i.i_category)(cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4))))) +------------------hashJoin[INNER_JOIN] hashCondition=((j.i_category = i.i_category))otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](s.ss_item_sk = i.i_item_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((s.ss_item_sk = i.i_item_sk))otherCondition=() ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](d.d_month_seq = date_dim.d_month_seq) +--------------------------hashJoin[INNER_JOIN] hashCondition=((d.d_month_seq = date_dim.d_month_seq))otherCondition=() ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](s.ss_sold_date_sk = d.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((s.ss_sold_date_sk = d.d_date_sk))otherCondition=() --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](c.c_customer_sk = s.ss_customer_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((c.c_customer_sk = s.ss_customer_sk))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](a.ca_address_sk = c.c_current_addr_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((a.ca_address_sk = c.c_current_addr_sk))otherCondition=() ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[customer] @@ -40,7 +40,7 @@ PhysicalResultSink ------------------------------------PhysicalDistribute --------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 3)) +------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) --------------------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out index b471b47aee1098..d7cbe916851c4f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out @@ -13,16 +13,16 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute -------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +------------------------------------filter((date_dim.d_moy = 8) and (date_dim.d_year = 2000)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute -----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((item.i_item_id = item.i_item_id))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute @@ -37,16 +37,16 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------PhysicalDistribute -------------------------------------filter((date_dim.d_moy = 8)(date_dim.d_year = 2000)) +------------------------------------filter((date_dim.d_moy = 8) and (date_dim.d_year = 2000)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute -----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((item.i_item_id = item.i_item_id))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute @@ -61,16 +61,16 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_bill_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------PhysicalOlapScan[web_sales] ----------------------------------PhysicalDistribute -------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +------------------------------------filter((date_dim.d_moy = 8) and (date_dim.d_year = 2000)) --------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute -----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((item.i_item_id = item.i_item_id))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute 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 e557ed58f537e7..18e0bab1d6f2d7 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 @@ -9,29 +9,29 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() --------------------PhysicalProject ----------------------filter((item.i_category = 'Jewelry')) ------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk))otherCondition=() --------------------------PhysicalProject ----------------------------filter((customer_address.ca_gmt_offset = -7.00)) ------------------------------PhysicalOlapScan[customer_address] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk))otherCondition=() --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[customer] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk))otherCondition=() ------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() ----------------------------------------------PhysicalProject ------------------------------------------------PhysicalOlapScan[store_sales] ----------------------------------------------PhysicalDistribute @@ -44,34 +44,34 @@ PhysicalResultSink ------------------------------------------------PhysicalOlapScan[promotion] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) +------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1999)) --------------------------------------------PhysicalOlapScan[date_dim] ----------PhysicalDistribute ------------hashAgg[GLOBAL] --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ----------------------PhysicalProject ------------------------filter((item.i_category = 'Jewelry')) --------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------filter((customer_address.ca_gmt_offset = -7.00)) --------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk))otherCondition=() ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[customer] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[store_sales] --------------------------------------------PhysicalDistribute @@ -80,6 +80,6 @@ PhysicalResultSink --------------------------------------------------PhysicalOlapScan[store] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) +--------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1999)) ----------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out index ec78b77b89e5f3..7970c74bb7fcf6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out @@ -8,16 +8,16 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) -------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_date_sk = date_dim.d_date_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[web_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq >= 1194)(date_dim.d_month_seq <= 1205)) +------------------------------filter((date_dim.d_month_seq <= 1205) and (date_dim.d_month_seq >= 1194)) --------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out index 2834c00bfaf8b8..573db03dabfb47 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out @@ -14,18 +14,18 @@ PhysicalResultSink ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ------------------------------------------PhysicalProject --------------------------------------------PhysicalOlapScan[store_sales] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) +----------------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('exportiunivamalg #9', 'scholaramalgamalg #14', 'scholaramalgamalg #7', 'scholaramalgamalg #9')) OR ((i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) ------------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out index ae54fc7949a9b9..9f860e83565907 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out @@ -5,20 +5,20 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalTopN --------PhysicalProject -----------hashJoin[INNER_JOIN](store.s_store_sk = sc.ss_store_sk) -------------hashJoin[INNER_JOIN](item.i_item_sk = sc.ss_item_sk) +----------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = sc.ss_store_sk))otherCondition=() +------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = sc.ss_item_sk))otherCondition=() --------------PhysicalDistribute -----------------hashJoin[INNER_JOIN](sb.ss_store_sk = sc.ss_store_sk)(cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE)) +----------------hashJoin[INNER_JOIN] hashCondition=((sb.ss_store_sk = sc.ss_store_sk))otherCondition=((cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE))) ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1232)(date_dim.d_month_seq >= 1221)) +--------------------------------filter((date_dim.d_month_seq <= 1232) and (date_dim.d_month_seq >= 1221)) ----------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------hashAgg[GLOBAL] @@ -29,12 +29,12 @@ PhysicalResultSink ------------------------------PhysicalDistribute --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq >= 1221)(date_dim.d_month_seq <= 1232)) +------------------------------------------filter((date_dim.d_month_seq <= 1232) and (date_dim.d_month_seq >= 1221)) --------------------------------------------PhysicalOlapScan[date_dim] --------------PhysicalDistribute ----------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out index d4d7f5452bd145..d1d08ca46b4c2c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out @@ -13,10 +13,10 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk))otherCondition=() ----------------------------------PhysicalOlapScan[web_sales] ----------------------------------PhysicalDistribute ------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) @@ -26,7 +26,7 @@ PhysicalResultSink ------------------------------------filter((date_dim.d_year = 1998)) --------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute ---------------------------------filter((cast(t_time as BIGINT) <= 77621)(time_dim.t_time >= 48821)) +--------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ----------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalOlapScan[warehouse] @@ -35,10 +35,10 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_time_sk = time_dim.t_time_sk) -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk))otherCondition=() ----------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------PhysicalDistribute ------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) @@ -48,7 +48,7 @@ PhysicalResultSink ------------------------------------filter((date_dim.d_year = 1998)) --------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute ---------------------------------filter((time_dim.t_time >= 48821)(cast(t_time as BIGINT) <= 77621)) +--------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ----------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out index 8161837a9572a8..0c95aeed5d20b6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out @@ -15,14 +15,14 @@ PhysicalResultSink ------------------------hashAgg[LOCAL] --------------------------PhysicalRepeat ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store_sales] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_month_seq >= 1206)(date_dim.d_month_seq <= 1217)) +----------------------------------------filter((date_dim.d_month_seq <= 1217) and (date_dim.d_month_seq >= 1206)) ------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out index 9a98918d915aa5..f0ae1101924d68 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out @@ -5,12 +5,12 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalTopN --------PhysicalProject -----------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk)( not (ca_city = bought_city)) +----------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = current_addr.ca_address_sk))otherCondition=(( not (ca_city = bought_city))) ------------PhysicalProject --------------PhysicalOlapScan[customer_address] ------------PhysicalDistribute --------------PhysicalProject -----------------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[customer] @@ -18,24 +18,24 @@ PhysicalResultSink --------------------PhysicalProject ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_dom >= 1)d_year IN (1998, 1999, 2000)(date_dim.d_dom <= 2)) +--------------------------------------------filter((date_dim.d_dom <= 2) and (date_dim.d_dom >= 1) and d_year IN (1998, 1999, 2000)) ----------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter(s_city IN ('Pleasant Hill', 'Five Points')) +------------------------------------------filter(s_city IN ('Five Points', 'Pleasant Hill')) --------------------------------------------PhysicalOlapScan[store] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out index 96ae616ceb1f84..1c43abde780f96 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out @@ -9,50 +9,50 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) +------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) +------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) --------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject -----------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) +----------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2000)(date_dim.d_moy <= 3)) +----------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ------------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk))otherCondition=() ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[customer_demographics] ----------------------------PhysicalDistribute -------------------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +------------------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk))otherCondition=() --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[web_sales] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) +------------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) --------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute -----------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[customer] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('TX', 'VA', 'MI')) +----------------------------------------filter(ca_state IN ('MI', 'TX', 'VA')) ------------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out index 279d0178403a46..e73080c00e2e50 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out @@ -8,17 +8,17 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk))otherCondition=() ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((customer_demographics.cd_gender = 'F')(customer_demographics.cd_marital_status = 'W')(customer_demographics.cd_education_status = 'College')) +--------------------------------filter((customer_demographics.cd_education_status = 'College') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'W')) ----------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out index 41ff62b38d65f7..ee6b5063371660 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out @@ -15,16 +15,16 @@ PhysicalResultSink ------------------------hashAgg[LOCAL] --------------------------PhysicalRepeat ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ---------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224)(d1.d_month_seq >= 1213)) +--------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) ----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute -----------------------------------hashJoin[LEFT_SEMI_JOIN](store.s_state = tmp1.s_state) +----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((store.s_state = tmp1.s_state))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store] @@ -37,13 +37,13 @@ PhysicalResultSink ------------------------------------------------PhysicalDistribute --------------------------------------------------hashAgg[LOCAL] ----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ---------------------------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((store.s_store_sk = store_sales.ss_store_sk))otherCondition=() +--------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() ----------------------------------------------------------PhysicalProject ------------------------------------------------------------PhysicalOlapScan[store_sales] ----------------------------------------------------------PhysicalDistribute ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------filter((date_dim.d_month_seq >= 1213)(date_dim.d_month_seq <= 1224)) +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) ----------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------------------PhysicalDistribute ----------------------------------------------------------PhysicalProject 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 4c1c5b51b79778..5967f0d4a2af9d 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 @@ -8,38 +8,38 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](tmp.time_sk = time_dim.t_time_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((tmp.time_sk = time_dim.t_time_sk))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](tmp.sold_item_sk = item.i_item_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((tmp.sold_item_sk = item.i_item_sk))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalUnion ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[web_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +--------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) ----------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +--------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) ----------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +--------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1998)) ----------------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject 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 04dc3a44d6815e..4f5be2c3f7e3fb 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 @@ -8,31 +8,31 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](warehouse.w_warehouse_sk = inventory.inv_warehouse_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((warehouse.w_warehouse_sk = inventory.inv_warehouse_sk))otherCondition=() ------------------PhysicalProject ---------------------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) +--------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = inventory.inv_item_sk) and (inventory.inv_date_sk = d2.d_date_sk))otherCondition=((inventory.inv_quantity_on_hand < catalog_sales.cs_quantity)) ----------------------PhysicalDistribute ------------------------PhysicalOlapScan[inventory] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](d1.d_week_seq = d2.d_week_seq) +--------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_week_seq = d2.d_week_seq))otherCondition=() ----------------------------PhysicalDistribute -------------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_returns.cr_item_sk = catalog_sales.cs_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) +------------------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((catalog_returns.cr_item_sk = catalog_sales.cs_item_sk) and (catalog_returns.cr_order_number = catalog_sales.cs_order_number))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[catalog_returns] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) +------------------------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk))otherCondition=() --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk))otherCondition=() ------------------------------------------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)) +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = d3.d_date_sk))otherCondition=((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) +--------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk))otherCondition=() ----------------------------------------------------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) +------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = d1.d_date_sk))otherCondition=() +--------------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() ----------------------------------------------------------PhysicalProject ------------------------------------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out index 0b350fd5018728..7a1b4590aff4c3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out @@ -5,30 +5,30 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalQuickSort --------PhysicalProject -----------hashJoin[INNER_JOIN](dj.ss_customer_sk = customer.c_customer_sk) +----------hashJoin[INNER_JOIN] hashCondition=((dj.ss_customer_sk = customer.c_customer_sk))otherCondition=() ------------PhysicalProject --------------PhysicalOlapScan[customer] ------------PhysicalDistribute ---------------filter((dj.cnt >= 1)(dj.cnt <= 5)) +--------------filter((dj.cnt <= 5) and (dj.cnt >= 1)) ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_dom >= 1)d_year IN (2000, 2001, 2002)(date_dim.d_dom <= 2)) +----------------------------------filter((date_dim.d_dom <= 2) and (date_dim.d_dom >= 1) and d_year IN (2000, 2001, 2002)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(hd_buy_potential IN ('501-1000', 'Unknown')(household_demographics.hd_vehicle_count > 0)(if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1)) +--------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1) and hd_buy_potential IN ('501-1000', 'Unknown')) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(s_county IN ('Fairfield County', 'Walker County', 'Daviess County', 'Barrow County')) +------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Fairfield County', 'Walker County')) --------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out index a4928f7851fefa..3a5637d8c3ba37 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out @@ -8,12 +8,12 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter(d_year IN (2000, 1999)) +------------------------filter(d_year IN (1999, 2000)) --------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject @@ -23,13 +23,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk))otherCondition=() ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------PhysicalOlapScan[web_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(d_year IN (2000, 1999)) +--------------------------filter(d_year IN (1999, 2000)) ----------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject @@ -39,23 +39,23 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(if((year_total > 0), (year_total / year_total), NULL) > if((year_total > 0), (year_total / year_total), NULL)) ---------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id) -----------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) +------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id))otherCondition=((if((year_total > 0), (year_total / year_total), NULL) > if((year_total > 0), (year_total / year_total), NULL))) +--------------hashJoin[INNER_JOIN] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id))otherCondition=() +----------------hashJoin[INNER_JOIN] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0)) +----------------------filter((t_s_firstyear.sale_type = 's') and (t_s_firstyear.year = 1999) and (t_s_firstyear.year_total > 0)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.year = 2000)) +----------------------filter((t_s_secyear.sale_type = 's') and (t_s_secyear.year = 2000)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w')) +--------------------filter((t_w_secyear.sale_type = 'w') and (t_w_secyear.year = 2000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0)(t_w_firstyear.sale_type = 'w')) +------------------filter((t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year = 1999) and (t_w_firstyear.year_total > 0)) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out index 96fe757f6e91a4..069dcb2c3c55b7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out @@ -9,12 +9,12 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------PhysicalUnion --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[catalog_returns] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = catalog_sales.cs_sold_date_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[catalog_sales] --------------------------PhysicalDistribute @@ -27,12 +27,12 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------PhysicalOlapScan[date_dim] --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[store_returns] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = store_sales.ss_item_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute @@ -45,12 +45,12 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------PhysicalOlapScan[date_dim] --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[web_returns] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = web_sales.ws_item_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[web_sales] --------------------------PhysicalDistribute @@ -66,7 +66,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](curr_yr.i_brand_id = prev_yr.i_brand_id)(curr_yr.i_class_id = prev_yr.i_class_id)(curr_yr.i_category_id = prev_yr.i_category_id)(curr_yr.i_manufact_id = prev_yr.i_manufact_id)((cast(cast(sales_cnt as DECIMALV3(17, 2)) as DECIMALV3(23, 8)) / cast(sales_cnt as DECIMALV3(17, 2))) < 0.900000) +------------hashJoin[INNER_JOIN] hashCondition=((curr_yr.i_brand_id = prev_yr.i_brand_id) and (curr_yr.i_category_id = prev_yr.i_category_id) and (curr_yr.i_class_id = prev_yr.i_class_id) and (curr_yr.i_manufact_id = prev_yr.i_manufact_id))otherCondition=(((cast(cast(sales_cnt as DECIMALV3(17, 2)) as DECIMALV3(23, 8)) / cast(sales_cnt as DECIMALV3(17, 2))) < 0.900000)) --------------PhysicalDistribute ----------------filter((curr_yr.d_year = 1999)) ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out index 86ed2d00f7c61f..8ed03212c33440 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out @@ -10,8 +10,8 @@ PhysicalResultSink --------------PhysicalUnion ----------------PhysicalDistribute ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ------------------------filter(ss_hdemo_sk IS NULL) --------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalDistribute @@ -20,8 +20,8 @@ PhysicalResultSink ------------------------PhysicalOlapScan[date_dim] ----------------PhysicalDistribute ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk))otherCondition=() ------------------------filter(ws_bill_addr_sk IS NULL) --------------------------PhysicalOlapScan[web_sales] ------------------------PhysicalDistribute @@ -30,8 +30,8 @@ PhysicalResultSink ------------------------PhysicalOlapScan[date_dim] ----------------PhysicalDistribute ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() ------------------------filter(cs_warehouse_sk IS NULL) --------------------------PhysicalOlapScan[catalog_sales] ------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out index b4978b34e97eee..3a46987274049b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out @@ -11,17 +11,17 @@ PhysicalResultSink ----------------PhysicalRepeat ------------------PhysicalUnion --------------------PhysicalProject -----------------------hashJoin[LEFT_OUTER_JOIN](ss.s_store_sk = sr.s_store_sk) +----------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((ss.s_store_sk = sr.s_store_sk))otherCondition=() ------------------------PhysicalProject --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalOlapScan[store_sales] --------------------------------------PhysicalDistribute -----------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) +----------------------------------------filter((date_dim.d_date <= 1998-09-04) and (date_dim.d_date >= 1998-08-05)) ------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[store] @@ -30,11 +30,11 @@ PhysicalResultSink ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_returns.sr_store_sk = store.s_store_sk) -------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_store_sk = store.s_store_sk))otherCondition=() +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalOlapScan[store_returns] --------------------------------------PhysicalDistribute -----------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) +----------------------------------------filter((date_dim.d_date <= 1998-09-04) and (date_dim.d_date >= 1998-08-05)) ------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[store] @@ -45,10 +45,10 @@ PhysicalResultSink ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------------PhysicalOlapScan[catalog_sales] ------------------------------------PhysicalDistribute ---------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +--------------------------------------filter((date_dim.d_date <= 1998-09-04) and (date_dim.d_date >= 1998-08-05)) ----------------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject @@ -56,23 +56,23 @@ PhysicalResultSink ------------------------------PhysicalDistribute --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalOlapScan[catalog_returns] --------------------------------------PhysicalDistribute -----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +----------------------------------------filter((date_dim.d_date <= 1998-09-04) and (date_dim.d_date >= 1998-08-05)) ------------------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject -----------------------hashJoin[LEFT_OUTER_JOIN](ws.wp_web_page_sk = wr.wp_web_page_sk) +----------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((ws.wp_web_page_sk = wr.wp_web_page_sk))otherCondition=() ------------------------PhysicalProject --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk))otherCondition=() +------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalOlapScan[web_sales] --------------------------------------PhysicalDistribute -----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +----------------------------------------filter((date_dim.d_date <= 1998-09-04) and (date_dim.d_date >= 1998-08-05)) ------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[web_page] @@ -81,11 +81,11 @@ PhysicalResultSink ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](web_returns.wr_web_page_sk = web_page.wp_web_page_sk) -------------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_web_page_sk = web_page.wp_web_page_sk))otherCondition=() +------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------PhysicalOlapScan[web_returns] --------------------------------------PhysicalDistribute -----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +----------------------------------------filter((date_dim.d_date <= 1998-09-04) and (date_dim.d_date >= 1998-08-05)) ------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalOlapScan[web_page] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out index 1eeff0d0be86e5..41b32615980e6b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out @@ -6,16 +6,16 @@ PhysicalResultSink ------PhysicalTopN --------PhysicalProject ----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) -------------hashJoin[LEFT_OUTER_JOIN](cs.cs_sold_year = ss.ss_sold_year)(cs.cs_item_sk = ss.ss_item_sk)(cs.cs_customer_sk = ss.ss_customer_sk) +------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((cs.cs_customer_sk = ss.ss_customer_sk) and (cs.cs_item_sk = ss.ss_item_sk) and (cs.cs_sold_year = ss.ss_sold_year))otherCondition=() --------------PhysicalProject -----------------hashJoin[LEFT_OUTER_JOIN](ws.ws_sold_year = ss.ss_sold_year)(ws.ws_item_sk = ss.ss_item_sk)(ws.ws_customer_sk = ss.ss_customer_sk) +----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((ws.ws_customer_sk = ss.ss_customer_sk) and (ws.ws_item_sk = ss.ss_item_sk) and (ws.ws_sold_year = ss.ss_sold_year))otherCondition=() ------------------PhysicalProject --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[LEFT_ANTI_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((store_returns.sr_ticket_number = store_sales.ss_ticket_number) and (store_sales.ss_item_sk = store_returns.sr_item_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute @@ -29,8 +29,8 @@ PhysicalResultSink ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[LEFT_ANTI_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((web_returns.wr_order_number = web_sales.ws_order_number) and (web_sales.ws_item_sk = web_returns.wr_item_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[web_sales] --------------------------------PhysicalDistribute @@ -44,8 +44,8 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[LEFT_ANTI_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((catalog_returns.cr_order_number = catalog_sales.cs_order_number) and (catalog_sales.cs_item_sk = catalog_returns.cr_item_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out index 05f8bd4c577563..2351430d3b597a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out @@ -5,21 +5,21 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalTopN --------PhysicalProject -----------hashJoin[INNER_JOIN](ms.ss_customer_sk = customer.c_customer_sk) +----------hashJoin[INNER_JOIN] hashCondition=((ms.ss_customer_sk = customer.c_customer_sk))otherCondition=() ------------PhysicalDistribute --------------PhysicalProject ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter(d_year IN (1998, 1999, 2000)(date_dim.d_dow = 1)) +----------------------------------filter((date_dim.d_dow = 1) and d_year IN (1998, 1999, 2000)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject @@ -27,7 +27,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((store.s_number_employees >= 200)(store.s_number_employees <= 295)) +------------------------------filter((store.s_number_employees <= 295) and (store.s_number_employees >= 200)) --------------------------------PhysicalOlapScan[store] ------------PhysicalDistribute --------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out index 8b1e9d5c7baa1b..2dc2925913e75e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out @@ -8,15 +8,15 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](expr_substring(s_zip, 1, 2) = expr_substring(ca_zip, 1, 2)) +----------------hashJoin[INNER_JOIN] hashCondition=((expr_substring(s_zip, 1, 2) = expr_substring(ca_zip, 1, 2)))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------PhysicalProject --------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1998)(date_dim.d_qoy = 2)) +----------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 1998)) ------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject @@ -26,7 +26,7 @@ PhysicalResultSink ----------------------PhysicalIntersect ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter(substring(ca_zip, 1, 5) IN ('47602', '16704', '35863', '28577', '83910', '36201', '58412', '48162', '28055', '41419', '80332', '38607', '77817', '24891', '16226', '18410', '21231', '59345', '13918', '51089', '20317', '17167', '54585', '67881', '78366', '47770', '18360', '51717', '73108', '14440', '21800', '89338', '45859', '65501', '34948', '25973', '73219', '25333', '17291', '10374', '18829', '60736', '82620', '41351', '52094', '19326', '25214', '54207', '40936', '21814', '79077', '25178', '75742', '77454', '30621', '89193', '27369', '41232', '48567', '83041', '71948', '37119', '68341', '14073', '16891', '62878', '49130', '19833', '24286', '27700', '40979', '50412', '81504', '94835', '84844', '71954', '39503', '57649', '18434', '24987', '12350', '86379', '27413', '44529', '98569', '16515', '27287', '24255', '21094', '16005', '56436', '91110', '68293', '56455', '54558', '10298', '83647', '32754', '27052', '51766', '19444', '13869', '45645', '94791', '57631', '20712', '37788', '41807', '46507', '21727', '71836', '81070', '50632', '88086', '63991', '20244', '31655', '51782', '29818', '63792', '68605', '94898', '36430', '57025', '20601', '82080', '33869', '22728', '35834', '29086', '92645', '98584', '98072', '11652', '78093', '57553', '43830', '71144', '53565', '18700', '90209', '71256', '38353', '54364', '28571', '96560', '57839', '56355', '50679', '45266', '84680', '34306', '34972', '48530', '30106', '15371', '92380', '84247', '92292', '68852', '13338', '34594', '82602', '70073', '98069', '85066', '47289', '11686', '98862', '26217', '47529', '63294', '51793', '35926', '24227', '14196', '24594', '32489', '99060', '49472', '43432', '49211', '14312', '88137', '47369', '56877', '20534', '81755', '15794', '12318', '21060', '73134', '41255', '63073', '81003', '73873', '66057', '51184', '51195', '45676', '92696', '70450', '90669', '98338', '25264', '38919', '59226', '58581', '60298', '17895', '19489', '52301', '80846', '95464', '68770', '51634', '19988', '18367', '18421', '11618', '67975', '25494', '41352', '95430', '15734', '62585', '97173', '33773', '10425', '75675', '53535', '17879', '41967', '12197', '67998', '79658', '59130', '72592', '14851', '43933', '68101', '50636', '25717', '71286', '24660', '58058', '72991', '95042', '15543', '33122', '69280', '11912', '59386', '27642', '65177', '17672', '33467', '64592', '36335', '54010', '18767', '63193', '42361', '49254', '33113', '33159', '36479', '59080', '11855', '81963', '31016', '49140', '29392', '41836', '32958', '53163', '13844', '73146', '23952', '65148', '93498', '14530', '46131', '58454', '13376', '13378', '83986', '12320', '17193', '59852', '46081', '98533', '52389', '13086', '68843', '31013', '13261', '60560', '13443', '45533', '83583', '11489', '58218', '19753', '22911', '25115', '86709', '27156', '32669', '13123', '51933', '39214', '41331', '66943', '14155', '69998', '49101', '70070', '35076', '14242', '73021', '59494', '15782', '29752', '37914', '74686', '83086', '34473', '15751', '81084', '49230', '91894', '60624', '17819', '28810', '63180', '56224', '39459', '55233', '75752', '43639', '55349', '86057', '62361', '50788', '31830', '58062', '18218', '85761', '60083', '45484', '21204', '90229', '70041', '41162', '35390', '16364', '39500', '68908', '26689', '52868', '81335', '40146', '11340', '61527', '61794', '71997', '30415', '59004', '29450', '58117', '69952', '33562', '83833', '27385', '61860', '96435', '48333', '23065', '32961', '84919', '61997', '99132', '22815', '56600', '68730', '48017', '95694', '32919', '88217', '27116', '28239', '58032', '18884', '16791', '21343', '97462', '18569', '75660', '15475')) +----------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) ------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalDistribute --------------------------PhysicalProject @@ -35,7 +35,7 @@ PhysicalResultSink --------------------------------PhysicalDistribute ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer_address] ----------------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out index 9c9f5b4ffe9bd0..b369178902bb06 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out @@ -15,15 +15,15 @@ PhysicalResultSink ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) +------------------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number))otherCondition=() --------------------------------PhysicalOlapScan[store_returns] ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk))otherCondition=() +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------PhysicalOlapScan[store_sales] ----------------------------------------PhysicalDistribute -------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) +------------------------------------------filter((date_dim.d_date <= 1998-09-27) and (date_dim.d_date >= 1998-08-28)) --------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------PhysicalDistribute ----------------------------------------filter((item.i_current_price > 50.00)) @@ -39,15 +39,15 @@ PhysicalResultSink ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) +------------------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number))otherCondition=() --------------------------------PhysicalOlapScan[catalog_returns] ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_catalog_page_sk = catalog_page.cp_catalog_page_sk) -----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) -------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_catalog_page_sk = catalog_page.cp_catalog_page_sk))otherCondition=() +----------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_promo_sk = promotion.p_promo_sk))otherCondition=() +------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk))otherCondition=() +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------------PhysicalDistribute -------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) +------------------------------------------filter((date_dim.d_date <= 1998-09-27) and (date_dim.d_date >= 1998-08-28)) --------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------PhysicalDistribute ----------------------------------------filter((item.i_current_price > 50.00)) @@ -63,17 +63,17 @@ PhysicalResultSink ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) +------------------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number))otherCondition=() --------------------------------PhysicalOlapScan[web_returns] --------------------------------PhysicalDistribute -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_site_sk = web_site.web_site_sk))otherCondition=() +------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = item.i_item_sk))otherCondition=() --------------------------------------PhysicalDistribute -----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_promo_sk = promotion.p_promo_sk) -------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_promo_sk = promotion.p_promo_sk))otherCondition=() +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------------PhysicalOlapScan[web_sales] --------------------------------------------PhysicalDistribute -----------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) +----------------------------------------------filter((date_dim.d_date <= 1998-09-27) and (date_dim.d_date >= 1998-08-28)) ------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------PhysicalDistribute --------------------------------------------filter((promotion.p_channel_tv = 'N')) 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 121a01f32e8a88..32ea0dcffdb2c6 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 @@ -6,10 +6,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------hashAgg[LOCAL] ----------PhysicalProject -------------hashJoin[INNER_JOIN](catalog_returns.cr_returning_addr_sk = customer_address.ca_address_sk) +------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_returning_addr_sk = customer_address.ca_address_sk))otherCondition=() --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[catalog_returns] --------------------PhysicalDistribute @@ -24,13 +24,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------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)) ---------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) +------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state))otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +--------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk))otherCondition=() ----------------PhysicalDistribute ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[customer] ----------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out index a6cb7d624913ec..b706b5bba24796 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out @@ -8,22 +8,22 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() ------------------PhysicalProject --------------------PhysicalOlapScan[store_sales] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) -------------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = inventory.inv_date_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((inventory.inv_item_sk = item.i_item_sk))otherCondition=() --------------------------PhysicalProject -----------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) +----------------------------filter((inventory.inv_quantity_on_hand <= 500) and (inventory.inv_quantity_on_hand >= 100)) ------------------------------PhysicalOlapScan[inventory] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((item.i_current_price >= 17.00)(item.i_current_price <= 47.00)i_manufact_id IN (138, 169, 339, 639)) +------------------------------filter((item.i_current_price <= 47.00) and (item.i_current_price >= 17.00) and i_manufact_id IN (138, 169, 339, 639)) --------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= 1999-09-07)(date_dim.d_date >= 1999-07-09)) +----------------------------filter((date_dim.d_date <= 1999-09-07) and (date_dim.d_date >= 1999-07-09)) ------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out index 1bda7e9b4c5ad9..affcd1c68b1ab8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out @@ -5,18 +5,18 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalTopN --------PhysicalProject -----------hashJoin[INNER_JOIN](sr_items.item_id = cr_items.item_id) +----------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = cr_items.item_id))otherCondition=() ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +----------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date))otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_item_sk = item.i_item_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_item_sk = item.i_item_sk))otherCondition=() ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[catalog_returns] @@ -28,23 +28,23 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +----------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------filter(d_date IN (2001-06-06, 2001-09-02, 2001-11-11)) ------------------------------------PhysicalOlapScan[date_dim] -------------hashJoin[INNER_JOIN](sr_items.item_id = wr_items.item_id) +------------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = wr_items.item_id))otherCondition=() --------------PhysicalProject ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date))otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) -------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = item.i_item_sk))otherCondition=() --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_returns] @@ -56,7 +56,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute @@ -68,12 +68,12 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date))otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = item.i_item_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = item.i_item_sk))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[web_returns] @@ -85,7 +85,7 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out index 3d5febe35de013..3cb79e15f36fc3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out @@ -5,20 +5,20 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalTopN --------PhysicalProject -----------hashJoin[INNER_JOIN](store_returns.sr_cdemo_sk = customer_demographics.cd_demo_sk) +----------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_cdemo_sk = customer_demographics.cd_demo_sk))otherCondition=() ------------PhysicalProject --------------PhysicalOlapScan[store_returns] ------------PhysicalDistribute --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk))otherCondition=() ------------------PhysicalProject --------------------PhysicalOlapScan[customer_demographics] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((household_demographics.hd_demo_sk = customer.c_current_hdemo_sk))otherCondition=() ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[customer] ------------------------------PhysicalDistribute @@ -26,11 +26,11 @@ PhysicalResultSink ----------------------------------filter((customer_address.ca_city = 'Oakwood')) ------------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](income_band.ib_income_band_sk = household_demographics.hd_income_band_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((income_band.ib_income_band_sk = household_demographics.hd_income_band_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((cast(ib_upper_bound as BIGINT) <= 55806)(income_band.ib_lower_bound >= 5806)) +--------------------------------filter((cast(ib_upper_bound as BIGINT) <= 55806) and (income_band.ib_lower_bound >= 5806)) ----------------------------------PhysicalOlapScan[income_band] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out index 24ab560b06c3f2..ec3fa8c91bed73 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out @@ -9,35 +9,35 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](reason.r_reason_sk = web_returns.wr_reason_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((reason.r_reason_sk = web_returns.wr_reason_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[reason] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[web_page] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)(cd1.cd_marital_status = cd2.cd_marital_status)(cd1.cd_education_status = cd2.cd_education_status) +------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_education_status = cd2.cd_education_status) and (cd1.cd_marital_status = cd2.cd_marital_status) and (cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[customer_demographics] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk))otherCondition=((((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00))))) --------------------------------------PhysicalProject ----------------------------------------filter(((((cd1.cd_marital_status = 'M') AND (cd1.cd_education_status = '4 yr Degree')) OR ((cd1.cd_marital_status = 'S') AND (cd1.cd_education_status = 'Secondary'))) OR ((cd1.cd_marital_status = 'W') AND (cd1.cd_education_status = 'Advanced Degree')))) ------------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------------PhysicalDistribute -----------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)(((ca_state IN ('FL', 'TX', 'DE') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('IN', 'ND', 'ID') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('MT', 'IL', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))) +----------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk))otherCondition=((((ca_state IN ('DE', 'FL', 'TX') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('ID', 'IN', 'ND') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('IL', 'MT', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))))) ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) +----------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_item_sk = web_returns.wr_item_sk) and (web_sales.ws_order_number = web_returns.wr_order_number))otherCondition=() ------------------------------------------------PhysicalProject --------------------------------------------------PhysicalOlapScan[web_returns] -------------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------------------------------PhysicalProject -----------------------------------------------------filter(((((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00)) OR ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00))) OR ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))((((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00)) OR ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00))) OR ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) +----------------------------------------------------filter(((((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00)) OR ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00))) OR ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00))) and ((((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00)) OR ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00))) OR ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) ------------------------------------------------------PhysicalOlapScan[web_sales] --------------------------------------------------PhysicalDistribute ----------------------------------------------------PhysicalProject @@ -45,6 +45,6 @@ PhysicalResultSink --------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------filter(((ca_state IN ('FL', 'TX', 'DE') OR ca_state IN ('IN', 'ND', 'ID')) OR ca_state IN ('MT', 'IL', 'OH'))(customer_address.ca_country = 'United States')) +----------------------------------------------filter(((ca_state IN ('DE', 'FL', 'TX') OR ca_state IN ('ID', 'IN', 'ND')) OR ca_state IN ('IL', 'MT', 'OH')) and (customer_address.ca_country = 'United States')) ------------------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out index 48fa4b29ff9ba8..d8b7351d0d160c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out @@ -15,15 +15,15 @@ PhysicalResultSink ------------------------hashAgg[LOCAL] --------------------------PhysicalRepeat ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = web_sales.ws_item_sk))otherCondition=() --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = web_sales.ws_sold_date_sk) +------------------------------------hashJoin[INNER_JOIN] hashCondition=((d1.d_date_sk = web_sales.ws_sold_date_sk))otherCondition=() --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[web_sales] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((d1.d_month_seq <= 1235)(d1.d_month_seq >= 1224)) +------------------------------------------filter((d1.d_month_seq <= 1235) and (d1.d_month_seq >= 1224)) --------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out index fcf6c2b80ac4c8..cf93a38df57838 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out @@ -10,15 +10,15 @@ PhysicalResultSink --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) +--------------------------------filter((date_dim.d_month_seq <= 1195) and (date_dim.d_month_seq >= 1184)) ----------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject @@ -27,15 +27,15 @@ PhysicalResultSink --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) +--------------------------------filter((date_dim.d_month_seq <= 1195) and (date_dim.d_month_seq >= 1184)) ----------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject @@ -44,15 +44,15 @@ PhysicalResultSink --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[web_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1195)(date_dim.d_month_seq >= 1184)) +--------------------------------filter((date_dim.d_month_seq <= 1195) and (date_dim.d_month_seq >= 1184)) ----------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out index e8b4b57dc82f4b..0240079d90f153 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out @@ -13,14 +13,14 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) +------------------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject @@ -35,14 +35,14 @@ PhysicalResultSink ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((time_dim.t_hour = 9)(time_dim.t_minute < 30)) +--------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute < 30)) ----------------------------------------PhysicalOlapScan[time_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject @@ -57,14 +57,14 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 9)) +------------------------------------filter((time_dim.t_hour = 9) and (time_dim.t_minute >= 30)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject @@ -79,14 +79,14 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((time_dim.t_hour = 10)(time_dim.t_minute < 30)) +----------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute < 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject @@ -101,14 +101,14 @@ PhysicalResultSink ----------------PhysicalDistribute ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 10)) +--------------------------------filter((time_dim.t_hour = 10) and (time_dim.t_minute >= 30)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject @@ -123,14 +123,14 @@ PhysicalResultSink --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((time_dim.t_minute < 30)(time_dim.t_hour = 11)) +------------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute < 30)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject @@ -145,14 +145,14 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk))otherCondition=() ------------------------PhysicalProject --------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((time_dim.t_hour = 11)(time_dim.t_minute >= 30)) +----------------------------filter((time_dim.t_hour = 11) and (time_dim.t_minute >= 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject @@ -167,14 +167,14 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((time_dim.t_hour = 12)(time_dim.t_minute < 30)) +--------------------------filter((time_dim.t_hour = 12) and (time_dim.t_minute < 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out index 4cb57595f362b6..db67dd551330b4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out @@ -15,15 +15,15 @@ PhysicalResultSink ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() +------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[store_sales] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter(((i_category IN ('Jewelry', 'Shoes', 'Electronics') AND i_class IN ('semi-precious', 'athletic', 'portable')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'rock', 'maternity')))) +------------------------------------------filter(((i_category IN ('Electronics', 'Jewelry', 'Shoes') AND i_class IN ('athletic', 'portable', 'semi-precious')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'maternity', 'rock')))) --------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out index b245af7262a6eb..5a9ea20b706661 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalDistribute ------------------------------------------hashAgg[LOCAL] --------------------------------------------PhysicalProject -----------------------------------------------filter((store_sales.ss_quantity <= 20)(store_sales.ss_quantity >= 1)) +----------------------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 1)) ------------------------------------------------PhysicalOlapScan[store_sales] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject @@ -35,7 +35,7 @@ PhysicalResultSink --------------------------------------PhysicalDistribute ----------------------------------------hashAgg[LOCAL] ------------------------------------------PhysicalProject ---------------------------------------------filter((store_sales.ss_quantity <= 20)(store_sales.ss_quantity >= 1)) +--------------------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 1)) ----------------------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalAssertNumRows @@ -43,7 +43,7 @@ PhysicalResultSink ------------------------------------PhysicalDistribute --------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------filter((store_sales.ss_quantity <= 20)(store_sales.ss_quantity >= 1)) +------------------------------------------filter((store_sales.ss_quantity <= 20) and (store_sales.ss_quantity >= 1)) --------------------------------------------PhysicalOlapScan[store_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalAssertNumRows @@ -51,7 +51,7 @@ PhysicalResultSink ----------------------------------PhysicalDistribute ------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------filter((store_sales.ss_quantity >= 21)(store_sales.ss_quantity <= 40)) +----------------------------------------filter((store_sales.ss_quantity <= 40) and (store_sales.ss_quantity >= 21)) ------------------------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalAssertNumRows @@ -59,7 +59,7 @@ PhysicalResultSink --------------------------------PhysicalDistribute ----------------------------------hashAgg[LOCAL] ------------------------------------PhysicalProject ---------------------------------------filter((store_sales.ss_quantity <= 40)(store_sales.ss_quantity >= 21)) +--------------------------------------filter((store_sales.ss_quantity <= 40) and (store_sales.ss_quantity >= 21)) ----------------------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalDistribute --------------------------PhysicalAssertNumRows @@ -67,7 +67,7 @@ PhysicalResultSink ------------------------------PhysicalDistribute --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------filter((store_sales.ss_quantity >= 21)(store_sales.ss_quantity <= 40)) +------------------------------------filter((store_sales.ss_quantity <= 40) and (store_sales.ss_quantity >= 21)) --------------------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalAssertNumRows @@ -75,7 +75,7 @@ PhysicalResultSink ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter((store_sales.ss_quantity <= 60)(store_sales.ss_quantity >= 41)) +----------------------------------filter((store_sales.ss_quantity <= 60) and (store_sales.ss_quantity >= 41)) ------------------------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute ----------------------PhysicalAssertNumRows @@ -83,7 +83,7 @@ PhysicalResultSink --------------------------PhysicalDistribute ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter((store_sales.ss_quantity <= 60)(store_sales.ss_quantity >= 41)) +--------------------------------filter((store_sales.ss_quantity <= 60) and (store_sales.ss_quantity >= 41)) ----------------------------------PhysicalOlapScan[store_sales] ------------------PhysicalDistribute --------------------PhysicalAssertNumRows @@ -91,7 +91,7 @@ PhysicalResultSink ------------------------PhysicalDistribute --------------------------hashAgg[LOCAL] ----------------------------PhysicalProject -------------------------------filter((store_sales.ss_quantity <= 60)(store_sales.ss_quantity >= 41)) +------------------------------filter((store_sales.ss_quantity <= 60) and (store_sales.ss_quantity >= 41)) --------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalDistribute ------------------PhysicalAssertNumRows @@ -99,7 +99,7 @@ PhysicalResultSink ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------filter((store_sales.ss_quantity <= 80)(store_sales.ss_quantity >= 61)) +----------------------------filter((store_sales.ss_quantity <= 80) and (store_sales.ss_quantity >= 61)) ------------------------------PhysicalOlapScan[store_sales] --------------PhysicalDistribute ----------------PhysicalAssertNumRows @@ -107,7 +107,7 @@ PhysicalResultSink --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter((store_sales.ss_quantity >= 61)(store_sales.ss_quantity <= 80)) +--------------------------filter((store_sales.ss_quantity <= 80) and (store_sales.ss_quantity >= 61)) ----------------------------PhysicalOlapScan[store_sales] ------------PhysicalDistribute --------------PhysicalAssertNumRows @@ -115,7 +115,7 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------filter((store_sales.ss_quantity <= 80)(store_sales.ss_quantity >= 61)) +------------------------filter((store_sales.ss_quantity <= 80) and (store_sales.ss_quantity >= 61)) --------------------------PhysicalOlapScan[store_sales] ----------PhysicalDistribute ------------PhysicalAssertNumRows @@ -123,7 +123,7 @@ PhysicalResultSink ----------------PhysicalDistribute ------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter((store_sales.ss_quantity >= 81)(store_sales.ss_quantity <= 100)) +----------------------filter((store_sales.ss_quantity <= 100) and (store_sales.ss_quantity >= 81)) ------------------------PhysicalOlapScan[store_sales] --------PhysicalDistribute ----------PhysicalAssertNumRows @@ -131,7 +131,7 @@ PhysicalResultSink --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------filter((store_sales.ss_quantity <= 100)(store_sales.ss_quantity >= 81)) +--------------------filter((store_sales.ss_quantity <= 100) and (store_sales.ss_quantity >= 81)) ----------------------PhysicalOlapScan[store_sales] ------PhysicalDistribute --------PhysicalAssertNumRows @@ -139,6 +139,6 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------filter((store_sales.ss_quantity >= 81)(store_sales.ss_quantity <= 100)) +------------------filter((store_sales.ss_quantity <= 100) and (store_sales.ss_quantity >= 81)) --------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out index 3a1416c9b94c38..90dd75d3de1713 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out @@ -9,20 +9,20 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) ---------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[web_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) +--------------------------------filter((web_page.wp_char_count <= 5200) and (web_page.wp_char_count >= 5000)) ----------------------------------PhysicalOlapScan[web_page] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((time_dim.t_hour >= 10)(time_dim.t_hour <= 11)) +------------------------------filter((time_dim.t_hour <= 11) and (time_dim.t_hour >= 10)) --------------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject @@ -33,20 +33,20 @@ PhysicalResultSink --------------PhysicalDistribute ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk) +--------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) -----------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((web_sales.ws_web_page_sk = web_page.wp_web_page_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[web_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) +----------------------------------filter((web_page.wp_char_count <= 5200) and (web_page.wp_char_count >= 5000)) ------------------------------------PhysicalOlapScan[web_page] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((time_dim.t_hour >= 16)(time_dim.t_hour <= 17)) +--------------------------------filter((time_dim.t_hour <= 17) and (time_dim.t_hour >= 16)) ----------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out index ecb3ba35ace5c0..c430d8c51c706a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out @@ -9,29 +9,29 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk))otherCondition=() --------------------PhysicalProject ----------------------filter((customer_address.ca_gmt_offset = -6.00)) ------------------------PhysicalOlapScan[customer_address] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk))otherCondition=() --------------------------PhysicalDistribute ----------------------------PhysicalProject ------------------------------filter((((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = 'Unknown')) OR ((customer_demographics.cd_marital_status = 'W') AND (customer_demographics.cd_education_status = 'Advanced Degree')))) --------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((household_demographics.hd_demo_sk = customer.c_current_hdemo_sk))otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returning_customer_sk = customer.c_customer_sk) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_returning_customer_sk = customer.c_customer_sk))otherCondition=() ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[customer] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk) +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk))otherCondition=() --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[catalog_returns] --------------------------------------------PhysicalDistribute @@ -39,7 +39,7 @@ PhysicalResultSink ------------------------------------------------PhysicalOlapScan[call_center] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2001)) +--------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ----------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out index af107f55b49ec4..b8e9891e4f1ac0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out @@ -12,8 +12,8 @@ PhysicalResultSink ------------------PhysicalQuickSort --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) ---------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((date_dim.d_date_sk = web_sales.ws_sold_date_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((item.i_item_sk = web_sales.ws_item_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[web_sales] ----------------------------PhysicalDistribute @@ -22,6 +22,6 @@ PhysicalResultSink ----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= 2002-05-27)(date_dim.d_date >= 2002-02-26)) +------------------------------filter((date_dim.d_date <= 2002-05-27) and (date_dim.d_date >= 2002-02-26)) --------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out index cc6eba640d6693..6acc7e3337f52c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out @@ -8,10 +8,10 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = store_sales.ss_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) +----------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = store_sales.ss_item_sk) and (store_returns.sr_ticket_number = store_sales.ss_ticket_number))otherCondition=() ------------------PhysicalProject --------------------PhysicalOlapScan[store_sales] -------------------hashJoin[INNER_JOIN](store_returns.sr_reason_sk = reason.r_reason_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_reason_sk = reason.r_reason_sk))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[store_returns] --------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out index 5fb17f0be15da4..48693b405c9647 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out @@ -9,18 +9,18 @@ PhysicalResultSink ------------hashAgg[GLOBAL] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws2.ws_order_number)( not (ws_warehouse_sk = ws_warehouse_sk)) +------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws2.ws_order_number))otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------PhysicalOlapScan[web_sales] ---------------------hashJoin[RIGHT_ANTI_JOIN](ws1.ws_order_number = wr1.wr_order_number) +--------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((ws1.ws_order_number = wr1.wr_order_number))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[web_returns] ----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) ---------------------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk))otherCondition=() +--------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[web_sales] ------------------------------PhysicalDistribute @@ -29,7 +29,7 @@ PhysicalResultSink ------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= 2000-04-01)(date_dim.d_date >= 2000-02-01)) +--------------------------------filter((date_dim.d_date <= 2000-04-01) and (date_dim.d_date >= 2000-02-01)) ----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out index 9d14722937730e..8044a1c9604b54 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------hashJoin[INNER_JOIN](ws1.ws_order_number = ws2.ws_order_number)( not (ws_warehouse_sk = ws_warehouse_sk)) +------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_order_number = ws2.ws_order_number))otherCondition=(( not (ws_warehouse_sk = ws_warehouse_sk))) --------PhysicalDistribute ----------PhysicalProject ------------PhysicalOlapScan[web_sales] @@ -19,13 +19,13 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------hashAgg[GLOBAL] ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws_wh.ws_order_number) +--------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = ws_wh.ws_order_number))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = web_returns.wr_order_number) +----------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((ws1.ws_order_number = web_returns.wr_order_number))otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_returns.wr_order_number = ws_wh.ws_order_number) +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_order_number = ws_wh.ws_order_number))otherCondition=() ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) @@ -33,9 +33,9 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[web_returns] ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) -----------------------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) -------------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_web_site_sk = web_site.web_site_sk))otherCondition=() +----------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_date_sk = date_dim.d_date_sk))otherCondition=() +------------------------------hashJoin[INNER_JOIN] hashCondition=((ws1.ws_ship_addr_sk = customer_address.ca_address_sk))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[web_sales] --------------------------------PhysicalDistribute @@ -44,7 +44,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date >= 1999-02-01)(date_dim.d_date <= 1999-04-02)) +----------------------------------filter((date_dim.d_date <= 1999-04-02) and (date_dim.d_date >= 1999-02-01)) ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out index 6d7df70b717438..42b27ad49bd1e5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out @@ -7,15 +7,15 @@ PhysicalResultSink --------PhysicalDistribute ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk))otherCondition=() ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk))otherCondition=() +--------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_time_sk = time_dim.t_time_sk))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) +--------------------------filter((time_dim.t_hour = 8) and (time_dim.t_minute >= 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out index 8b918462450d96..9ba62fe3fa0a36 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out @@ -7,29 +7,29 @@ PhysicalResultSink --------PhysicalDistribute ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[FULL_OUTER_JOIN](ssci.customer_sk = csci.customer_sk)(ssci.item_sk = csci.item_sk) +--------------hashJoin[FULL_OUTER_JOIN] hashCondition=((ssci.customer_sk = csci.customer_sk) and (ssci.item_sk = csci.item_sk))otherCondition=() ----------------PhysicalProject ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[store_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) +--------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ----------------------------------PhysicalOlapScan[date_dim] ----------------PhysicalProject ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) +--------------------------------filter((date_dim.d_month_seq <= 1225) and (date_dim.d_month_seq >= 1214)) ----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out index 0d4d9f7e8a8c61..ee9ef5c16e49d0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out @@ -12,17 +12,17 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk))otherCondition=() --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_date <= 2002-06-19)(date_dim.d_date >= 2002-05-20)) +----------------------------------filter((date_dim.d_date <= 2002-06-19) and (date_dim.d_date >= 2002-05-20)) ------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(i_category IN ('Sports', 'Music', 'Shoes')) +------------------------------filter(i_category IN ('Music', 'Shoes', 'Sports')) --------------------------------PhysicalOlapScan[item] 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 919fdb60190fba..ef4fd9d0b9522b 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,16 +8,16 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk) +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk))otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_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) +--------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk))otherCondition=() +----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk))otherCondition=() +------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_ship_date_sk = date_dim.d_date_sk))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[catalog_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq >= 1224)(date_dim.d_month_seq <= 1235)) +------------------------------filter((date_dim.d_month_seq <= 1235) and (date_dim.d_month_seq >= 1224)) --------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out index d71a0ef61c5f05..504b076849a3ea 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out @@ -8,19 +8,19 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +----------------hashJoin[INNER_JOIN] hashCondition=((lineitem.l_orderkey = orders.o_orderkey))otherCondition=() ------------------PhysicalProject --------------------filter((lineitem.l_returnflag = 'R')) ----------------------PhysicalOlapScan[lineitem] ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) +--------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_nationkey = nation.n_nationkey))otherCondition=() ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_custkey = orders.o_custkey))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[customer] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) +------------------------------filter((orders.o_orderdate < 1994-01-01) and (orders.o_orderdate >= 1993-10-01)) --------------------------------PhysicalOlapScan[orders] ----------------------PhysicalDistribute ------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out index 1d1e947be2d045..ead04e26a783c7 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out @@ -8,11 +8,11 @@ PhysicalResultSink ----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * cast(ps_availqty as DECIMALV3(10, 0)))) * 0.000002) as DOUBLE)) ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +----------------hashJoin[INNER_JOIN] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey))otherCondition=() ------------------PhysicalProject --------------------PhysicalOlapScan[partsupp] ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[supplier] ----------------------PhysicalDistribute @@ -25,11 +25,11 @@ PhysicalResultSink ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +------------------------hashJoin[INNER_JOIN] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey))otherCondition=() --------------------------PhysicalProject ----------------------------PhysicalOlapScan[partsupp] --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +----------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey))otherCondition=() ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[supplier] ------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out index 1fec7c6597fd30..3f7f5677881a35 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out @@ -8,10 +8,10 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +----------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = lineitem.l_orderkey))otherCondition=() ------------------PhysicalProject --------------------PhysicalOlapScan[orders] ------------------PhysicalProject ---------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) +--------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < 1995-01-01) and (lineitem.l_receiptdate >= 1994-01-01) and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) ----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out index ab3e4ce172b9db..81a0302dae9798 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out @@ -10,7 +10,7 @@ PhysicalResultSink --------------PhysicalProject ----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) +--------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((customer.c_custkey = orders.o_custkey))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------filter(( not (o_comment like '%special%requests%'))) diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out index 747b102ab8889a..1760d9b6758ab3 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out @@ -6,11 +6,11 @@ PhysicalResultSink ------PhysicalDistribute --------hashAgg[LOCAL] ----------PhysicalProject -------------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) +------------hashJoin[INNER_JOIN] hashCondition=((lineitem.l_partkey = part.p_partkey))otherCondition=() --------------PhysicalProject ----------------PhysicalOlapScan[part] --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) +------------------filter((lineitem.l_shipdate < 1995-10-01) and (lineitem.l_shipdate >= 1995-09-01)) --------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out index ff4350e0806069..fbdc8c9187016b 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out @@ -5,16 +5,16 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalQuickSort --------PhysicalProject -----------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) +----------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = revenue0.supplier_no))otherCondition=() ------------PhysicalProject --------------PhysicalOlapScan[supplier] ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) +--------------hashJoin[INNER_JOIN] hashCondition=((revenue0.total_revenue = max(total_revenue)))otherCondition=() ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute --------------------hashAgg[LOCAL] ----------------------PhysicalProject -------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +------------------------filter((lineitem.l_shipdate < 1996-04-01) and (lineitem.l_shipdate >= 1996-01-01)) --------------------------PhysicalOlapScan[lineitem] ----------------PhysicalDistribute ------------------hashAgg[GLOBAL] @@ -25,6 +25,6 @@ PhysicalResultSink ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +----------------------------------filter((lineitem.l_shipdate < 1996-04-01) and (lineitem.l_shipdate >= 1996-01-01)) ------------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out index e2b58de0eb720c..4d2cf07573f975 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out @@ -8,13 +8,13 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +----------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey))otherCondition=() ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +--------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = partsupp.ps_partkey))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[partsupp] ----------------------PhysicalProject -------------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) +------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) --------------------------PhysicalOlapScan[part] ------------------PhysicalDistribute --------------------PhysicalProject diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out index 65bad6d59809a4..1c481f1106f8d4 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out @@ -11,11 +11,11 @@ PhysicalResultSink ----------------PhysicalQuickSort ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = lineitem.l_partkey))otherCondition=() ------------------------PhysicalProject --------------------------PhysicalOlapScan[lineitem] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) +----------------------------filter((part.p_brand = 'Brand#23') and (part.p_container = 'MED BOX')) ------------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out index ae401ef1cd429f..0f434ae5876fa2 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out @@ -6,16 +6,16 @@ PhysicalResultSink ------PhysicalTopN --------hashAgg[LOCAL] ----------PhysicalProject -------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = lineitem.l_orderkey))otherCondition=() --------------PhysicalProject ----------------PhysicalOlapScan[lineitem] --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_custkey = orders.o_custkey))otherCondition=() --------------------PhysicalProject ----------------------PhysicalOlapScan[customer] --------------------PhysicalDistribute -----------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) +----------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((orders.o_orderkey = lineitem.l_orderkey))otherCondition=() ------------------------PhysicalProject --------------------------PhysicalOlapScan[orders] ------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out index a5bc461b44c791..68b8bf42862ce1 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out @@ -5,12 +5,12 @@ PhysicalResultSink ----PhysicalDistribute ------hashAgg[LOCAL] --------PhysicalProject -----------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) +----------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = lineitem.l_partkey))otherCondition=(((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15)))) ------------PhysicalProject ---------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) +--------------filter(((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) and (lineitem.l_shipinstruct = 'DELIVER IN PERSON') and l_shipmode IN ('AIR REG', 'AIR')) ----------------PhysicalOlapScan[lineitem] ------------PhysicalDistribute --------------PhysicalProject -----------------filter((part.p_size >= 1)(((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))) +----------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15))) and (part.p_size >= 1)) ------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out index 4c8fc44b7c203f..a77e4d2b8043cb 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out @@ -10,21 +10,21 @@ PhysicalResultSink --------------PhysicalQuickSort ----------------PhysicalDistribute ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) +--------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = partsupp.ps_suppkey))otherCondition=() ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +--------------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = partsupp.ps_partkey))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[partsupp] ----------------------------PhysicalProject -------------------------------filter((part.p_size = 15)(p_type like '%BRASS')) +------------------------------filter((p_type like '%BRASS') and (part.p_size = 15)) --------------------------------PhysicalOlapScan[part] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey))otherCondition=() ----------------------------PhysicalOlapScan[supplier] ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) +------------------------------hashJoin[INNER_JOIN] hashCondition=((nation.n_regionkey = region.r_regionkey))otherCondition=() --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out index 300ef0d9d617ea..91e1da666d32cc 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out @@ -5,26 +5,26 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalQuickSort --------PhysicalProject -----------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = t3.ps_suppkey) +----------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((supplier.s_suppkey = t3.ps_suppkey))otherCondition=() ------------PhysicalDistribute --------------PhysicalProject -----------------hashJoin[INNER_JOIN](t2.l_partkey = t1.ps_partkey)(t2.l_suppkey = t1.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > t2.l_q) +----------------hashJoin[INNER_JOIN] hashCondition=((t2.l_partkey = t1.ps_partkey) and (t2.l_suppkey = t1.ps_suppkey))otherCondition=((cast(ps_availqty as DECIMALV3(38, 3)) > t2.l_q)) ------------------PhysicalProject --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) +----------------------------filter((lineitem.l_shipdate < 1995-01-01) and (lineitem.l_shipdate >= 1994-01-01)) ------------------------------PhysicalOlapScan[lineitem] ------------------PhysicalDistribute ---------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) +--------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((partsupp.ps_partkey = part.p_partkey))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[partsupp] ----------------------PhysicalProject ------------------------filter((p_name like 'forest%')) --------------------------PhysicalOlapScan[part] ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey))otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[supplier] ----------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out index 9913e27f5fb37b..98434f097634ac 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out @@ -5,25 +5,25 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalQuickSort --------PhysicalProject -----------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) +----------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((supplier.s_suppkey = partsupp.ps_suppkey))otherCondition=() ------------PhysicalDistribute --------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) +----------------hashJoin[INNER_JOIN] hashCondition=((lineitem.l_partkey = partsupp.ps_partkey) and (lineitem.l_suppkey = partsupp.ps_suppkey))otherCondition=((cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity)))) ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) +--------------------------filter((lineitem.l_shipdate < 1995-01-01) and (lineitem.l_shipdate >= 1994-01-01)) ----------------------------PhysicalOlapScan[lineitem] ------------------PhysicalDistribute ---------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) +--------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((partsupp.ps_partkey = part.p_partkey))otherCondition=() ----------------------PhysicalProject ------------------------PhysicalOlapScan[partsupp] ----------------------PhysicalProject ------------------------filter((p_name like 'forest%')) --------------------------PhysicalOlapScan[part] ------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey))otherCondition=() ----------------PhysicalProject ------------------PhysicalOlapScan[supplier] ----------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out index 2f4348619b2373..8cf86dcf6e74d0 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out @@ -8,23 +8,23 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) +----------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = l1.l_orderkey))otherCondition=() ------------------PhysicalProject --------------------filter((orders.o_orderstatus = 'F')) ----------------------PhysicalOlapScan[orders] -------------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) +------------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((l2.l_orderkey = l1.l_orderkey))otherCondition=(( not (l_suppkey = l_suppkey))) --------------------PhysicalProject ----------------------PhysicalOlapScan[lineitem] ---------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) +--------------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((l3.l_orderkey = l1.l_orderkey))otherCondition=(( not (l_suppkey = l_suppkey))) ----------------------PhysicalProject ------------------------filter((l3.l_receiptdate > l3.l_commitdate)) --------------------------PhysicalOlapScan[lineitem] -----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = l1.l_suppkey))otherCondition=() ------------------------PhysicalProject --------------------------filter((l1.l_receiptdate > l1.l_commitdate)) ----------------------------PhysicalOlapScan[lineitem] ------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[supplier] ----------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out index 7845eba2baf8c4..a0e1e755bd2783 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out @@ -8,20 +8,20 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) +----------------hashJoin[RIGHT_ANTI_JOIN] hashCondition=((orders.o_custkey = customer.c_custkey))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[orders] ------------------PhysicalDistribute --------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) --------------------------PhysicalOlapScan[customer] ----------------------PhysicalDistribute ------------------------hashAgg[GLOBAL] --------------------------PhysicalDistribute ----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +--------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ----------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out index 04cec030a0f017..d4af45578fb959 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out @@ -6,13 +6,13 @@ PhysicalResultSink ------PhysicalTopN --------hashAgg[LOCAL] ----------PhysicalProject -------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------hashJoin[INNER_JOIN] hashCondition=((lineitem.l_orderkey = orders.o_orderkey))otherCondition=() --------------PhysicalProject ----------------filter((lineitem.l_shipdate > 1995-03-15)) ------------------PhysicalOlapScan[lineitem] --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_custkey = orders.o_custkey))otherCondition=() --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((orders.o_orderdate < 1995-03-15)) diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out index 4d9d91d96323ae..5155b3d85d378a 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out @@ -8,11 +8,11 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) +----------------hashJoin[RIGHT_SEMI_JOIN] hashCondition=((lineitem.l_orderkey = orders.o_orderkey))otherCondition=() ------------------PhysicalProject --------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) ----------------------PhysicalOlapScan[lineitem] ------------------PhysicalProject ---------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) +--------------------filter((orders.o_orderdate < 1993-10-01) and (orders.o_orderdate >= 1993-07-01)) ----------------------PhysicalOlapScan[orders] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out index e73b6cac53c7a2..0835f630e33d3f 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out @@ -8,20 +8,20 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) +----------------hashJoin[INNER_JOIN] hashCondition=((customer.c_custkey = orders.o_custkey) and (customer.c_nationkey = supplier.s_nationkey))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((lineitem.l_orderkey = orders.o_orderkey))otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) +--------------------------hashJoin[INNER_JOIN] hashCondition=((lineitem.l_suppkey = supplier.s_suppkey))otherCondition=() ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[lineitem] ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +------------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[supplier] --------------------------------PhysicalDistribute -----------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((nation.n_regionkey = region.r_regionkey))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[nation] @@ -30,7 +30,7 @@ PhysicalResultSink ----------------------------------------filter((region.r_name = 'ASIA')) ------------------------------------------PhysicalOlapScan[region] ------------------------PhysicalProject ---------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) +--------------------------filter((orders.o_orderdate < 1995-01-01) and (orders.o_orderdate >= 1994-01-01)) ----------------------------PhysicalOlapScan[orders] ------------------PhysicalDistribute --------------------PhysicalProject diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out index 0cd7c1f4b2b411..71385b8a5fd9f5 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out @@ -5,6 +5,6 @@ PhysicalResultSink ----PhysicalDistribute ------hashAgg[LOCAL] --------PhysicalProject -----------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) +----------filter((lineitem.l_discount <= 0.07) and (lineitem.l_discount >= 0.05) and (lineitem.l_quantity < 24.00) and (lineitem.l_shipdate < 1995-01-01) and (lineitem.l_shipdate >= 1994-01-01)) ------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out index 2eefbd562eb216..c1c28159100da4 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out @@ -8,21 +8,21 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) +----------------hashJoin[INNER_JOIN] hashCondition=((customer.c_custkey = orders.o_custkey) and (customer.c_nationkey = n2.n_nationkey))otherCondition=() ------------------PhysicalProject --------------------PhysicalOlapScan[customer] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = lineitem.l_orderkey))otherCondition=() ------------------------PhysicalProject --------------------------PhysicalOlapScan[orders] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +--------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = lineitem.l_suppkey))otherCondition=() ----------------------------PhysicalProject -------------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) +------------------------------filter((lineitem.l_shipdate <= 1996-12-31) and (lineitem.l_shipdate >= 1995-01-01)) --------------------------------PhysicalOlapScan[lineitem] ----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) +------------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = n1.n_nationkey))otherCondition=() --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[supplier] --------------------------------PhysicalDistribute 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 e699fc97c2bd55..5e96e7b598c0ed 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 @@ -9,27 +9,27 @@ PhysicalResultSink ------------PhysicalDistribute --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) +------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = n2.n_nationkey))otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = lineitem.l_suppkey))otherCondition=() ------------------------PhysicalProject --------------------------PhysicalOlapScan[supplier] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +----------------------------hashJoin[INNER_JOIN] hashCondition=((lineitem.l_orderkey = orders.o_orderkey))otherCondition=() ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_custkey = customer.c_custkey))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) +----------------------------------------filter((orders.o_orderdate <= 1996-12-31) and (orders.o_orderdate >= 1995-01-01)) ------------------------------------------PhysicalOlapScan[orders] ------------------------------------PhysicalDistribute ---------------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_nationkey = n1.n_nationkey))otherCondition=() ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] ----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) +------------------------------------------hashJoin[INNER_JOIN] hashCondition=((n1.n_regionkey = region.r_regionkey))otherCondition=() --------------------------------------------PhysicalDistribute ----------------------------------------------PhysicalProject ------------------------------------------------PhysicalOlapScan[nation] @@ -39,7 +39,7 @@ PhysicalResultSink --------------------------------------------------PhysicalOlapScan[region] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = lineitem.l_partkey))otherCondition=() ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[lineitem] ------------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out index 83214427047d2f..3f330af3959550 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out @@ -8,18 +8,18 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) +----------------hashJoin[INNER_JOIN] hashCondition=((partsupp.ps_partkey = lineitem.l_partkey) and (partsupp.ps_suppkey = lineitem.l_suppkey))otherCondition=() ------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +----------------------hashJoin[INNER_JOIN] hashCondition=((orders.o_orderkey = lineitem.l_orderkey))otherCondition=() ------------------------PhysicalProject --------------------------PhysicalOlapScan[orders] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +----------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_suppkey = lineitem.l_suppkey))otherCondition=() ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------------------hashJoin[INNER_JOIN] hashCondition=((part.p_partkey = lineitem.l_partkey))otherCondition=() ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------PhysicalOlapScan[lineitem] @@ -28,7 +28,7 @@ PhysicalResultSink ----------------------------------------filter((p_name like '%green%')) ------------------------------------------PhysicalOlapScan[part] ------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------------hashJoin[INNER_JOIN] hashCondition=((supplier.s_nationkey = nation.n_nationkey))otherCondition=() ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[supplier] ----------------------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out deleted file mode 100644 index 7740edd76cc3a5..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out +++ /dev/null @@ -1,13 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((lineitem.l_shipdate <= 1998-09-02)) -------------------PhysicalOlapScan[lineitem] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out deleted file mode 100644 index d71a0ef61c5f05..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out +++ /dev/null @@ -1,28 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -------------------PhysicalProject ---------------------filter((lineitem.l_returnflag = 'R')) -----------------------PhysicalOlapScan[lineitem] -------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) ---------------------------------PhysicalOlapScan[orders] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[nation] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out deleted file mode 100644 index 1d1e947be2d045..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out +++ /dev/null @@ -1,39 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * cast(ps_availqty as DECIMALV3(10, 0)))) * 0.000002) as DOUBLE)) -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[partsupp] -------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[supplier] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((nation.n_name = 'GERMANY')) -----------------------------PhysicalOlapScan[nation] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[partsupp] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((nation.n_name = 'GERMANY')) -------------------------------------PhysicalOlapScan[nation] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out deleted file mode 100644 index 1fec7c6597fd30..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out +++ /dev/null @@ -1,17 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[orders] -------------------PhysicalProject ---------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) -----------------------PhysicalOlapScan[lineitem] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out deleted file mode 100644 index ab3e4ce172b9db..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out +++ /dev/null @@ -1,21 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter(( not (o_comment like '%special%requests%'))) -----------------------------PhysicalOlapScan[orders] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out deleted file mode 100644 index 747b102ab8889a..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out +++ /dev/null @@ -1,16 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalProject -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) ---------------PhysicalProject -----------------PhysicalOlapScan[part] ---------------PhysicalDistribute -----------------PhysicalProject -------------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) ---------------------PhysicalOlapScan[lineitem] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out deleted file mode 100644 index ff4350e0806069..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out +++ /dev/null @@ -1,30 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) -------------PhysicalProject ---------------PhysicalOlapScan[supplier] -------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) ---------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) -------------------------------------PhysicalOlapScan[lineitem] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out deleted file mode 100644 index e2b58de0eb720c..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out +++ /dev/null @@ -1,23 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[partsupp] -----------------------PhysicalProject -------------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) ---------------------------PhysicalOlapScan[part] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out deleted file mode 100644 index 65bad6d59809a4..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out +++ /dev/null @@ -1,21 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalProject -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(l_quantity as DECIMALV3(38, 5)) < (0.2 * avg(cast(l_quantity as DECIMALV3(17, 4))) OVER(PARTITION BY p_partkey)))) ---------------PhysicalWindow -----------------PhysicalQuickSort -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[lineitem] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) -------------------------------PhysicalOlapScan[part] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out deleted file mode 100644 index ae401ef1cd429f..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out +++ /dev/null @@ -1,26 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) ---------------PhysicalProject -----------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[customer] ---------------------PhysicalDistribute -----------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[orders] -------------------------PhysicalProject ---------------------------filter((sum(l_quantity) > 300.00)) -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineitem] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out deleted file mode 100644 index a5bc461b44c791..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out +++ /dev/null @@ -1,16 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) -------------PhysicalProject ---------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) -----------------PhysicalOlapScan[lineitem] -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((part.p_size >= 1)(((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))) -------------------PhysicalOlapScan[part] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out deleted file mode 100644 index 4c8fc44b7c203f..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out +++ /dev/null @@ -1,35 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------filter((partsupp.ps_supplycost = min(ps_supplycost) OVER(PARTITION BY p_partkey))) -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[partsupp] -----------------------------PhysicalProject -------------------------------filter((part.p_size = 15)(p_type like '%BRASS')) ---------------------------------PhysicalOlapScan[part] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -----------------------------PhysicalOlapScan[supplier] -----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((region.r_name = 'EUROPE')) ---------------------------------------PhysicalOlapScan[region] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out deleted file mode 100644 index 9913e27f5fb37b..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out +++ /dev/null @@ -1,33 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) -----------------------------PhysicalOlapScan[lineitem] -------------------PhysicalDistribute ---------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[partsupp] -----------------------PhysicalProject -------------------------filter((p_name like 'forest%')) ---------------------------PhysicalOlapScan[part] -------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -----------------PhysicalProject -------------------PhysicalOlapScan[supplier] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((nation.n_name = 'CANADA')) -----------------------PhysicalOlapScan[nation] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out deleted file mode 100644 index 52f7b7a7ee0439..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out +++ /dev/null @@ -1,35 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) -------------------PhysicalProject ---------------------filter((orders.o_orderstatus = 'F')) -----------------------PhysicalOlapScan[orders] -------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) -----------------------PhysicalProject -------------------------PhysicalOlapScan[lineitem] -----------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) -------------------------PhysicalProject ---------------------------filter((l3.l_receiptdate > l3.l_commitdate)) -----------------------------PhysicalOlapScan[lineitem] -------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) ---------------------------PhysicalProject -----------------------------filter((l1.l_receiptdate > l1.l_commitdate)) -------------------------------PhysicalOlapScan[lineitem] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((nation.n_name = 'SAUDI ARABIA')) -------------------------------------PhysicalOlapScan[nation] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out deleted file mode 100644 index 7845eba2baf8c4..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out +++ /dev/null @@ -1,27 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[orders] -------------------PhysicalDistribute ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) -----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalDistribute -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) -----------------------------------PhysicalOlapScan[customer] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out deleted file mode 100644 index 04cec030a0f017..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out +++ /dev/null @@ -1,24 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) ---------------PhysicalProject -----------------filter((lineitem.l_shipdate > 1995-03-15)) -------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1995-03-15)) ---------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((customer.c_mktsegment = 'BUILDING')) ---------------------------PhysicalOlapScan[customer] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out deleted file mode 100644 index 4d9d91d96323ae..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out +++ /dev/null @@ -1,18 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) -----------------------PhysicalOlapScan[lineitem] -------------------PhysicalProject ---------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) -----------------------PhysicalOlapScan[orders] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out deleted file mode 100644 index e73b6cac53c7a2..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out +++ /dev/null @@ -1,38 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[lineitem] -----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[supplier] ---------------------------------PhysicalDistribute -----------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[nation] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------filter((region.r_name = 'ASIA')) -------------------------------------------PhysicalOlapScan[region] -------------------------PhysicalProject ---------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) -----------------------------PhysicalOlapScan[orders] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[customer] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out deleted file mode 100644 index 0cd7c1f4b2b411..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out +++ /dev/null @@ -1,10 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) -------------PhysicalOlapScan[lineitem] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out deleted file mode 100644 index 2eefbd562eb216..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out +++ /dev/null @@ -1,37 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[orders] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -----------------------------PhysicalProject -------------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) ---------------------------------PhysicalOlapScan[lineitem] -----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[supplier] ---------------------------------PhysicalDistribute -----------------------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) -------------------------------------PhysicalProject ---------------------------------------filter(((n1.n_name = 'FRANCE') OR (n1.n_name = 'GERMANY'))) -----------------------------------------PhysicalOlapScan[nation] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------filter(((n2.n_name = 'GERMANY') OR (n2.n_name = 'FRANCE'))) -------------------------------------------PhysicalOlapScan[nation] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out deleted file mode 100644 index 07a9cd9ca20154..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out +++ /dev/null @@ -1,49 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -------------------------PhysicalProject ---------------------------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) ---------------------------------------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) -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[lineitem] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) -----------------------------------------------------PhysicalOlapScan[part] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[nation] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((region.r_name = 'AMERICA')) -------------------------------------PhysicalOlapScan[region] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[nation] - diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out deleted file mode 100644 index 83214427047d2f..00000000000000 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out +++ /dev/null @@ -1,40 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -PhysicalResultSink ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[orders] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[lineitem] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------filter((p_name like '%green%')) -------------------------------------------PhysicalOlapScan[part] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[supplier] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[nation] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] - diff --git a/regression-test/data/query_p0/sql_functions/array_functions/test_array_with_scale_type.out b/regression-test/data/query_p0/sql_functions/array_functions/test_array_with_scale_type.out index c20f9977c3eba3..70feafa910b031 100644 --- a/regression-test/data/query_p0/sql_functions/array_functions/test_array_with_scale_type.out +++ b/regression-test/data/query_p0/sql_functions/array_functions/test_array_with_scale_type.out @@ -6,8 +6,8 @@ 2022-12-02T22:23:25 -- !select -- -2022-12-01T22:23:24.999 -2022-12-02T22:23:24.999 +2022-12-01T22:23:25 +2022-12-02T22:23:25 -- !select -- 2022-12-02T22:23:24.999999 @@ -16,8 +16,8 @@ 2022-12-02T22:23:25 -- !select -- -2022-12-01T23:23:24.999 -2022-12-02T23:23:24.999 +2022-12-01T23:23:25 +2022-12-02T23:23:25 -- !select -- 23 @@ -76,8 +76,8 @@ [24.990, 25.990] -- !select -- -[2022-12-01 22:23:24.999] -[2022-12-02 22:23:24.999] +[2022-12-01 22:23:25.000] +[2022-12-02 22:23:25.000] -- !select -- [2022-12-02 22:23:25.000, 2022-12-02 22:23:23.998] @@ -91,8 +91,8 @@ [] -- !select -- -[2022-12-01 22:23:24.999, 2022-12-01 23:23:24.999] -[2022-12-02 22:23:24.999, 2022-12-02 23:23:24.999] +[2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000] +[2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000] -- !select -- \N @@ -115,44 +115,44 @@ [2022-12-02 22:23:24.999, 2022-12-02 22:23:23.997] -- !select -- -[2022-12-01 22:23:24.999, 2022-12-01 23:23:24.999] -[2022-12-02 22:23:24.999, 2022-12-02 23:23:24.999] +[2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000] +[2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000] -- !select -- -[2022-12-01 22:23:24.999, 2022-12-01 23:23:24.999, 2022-12-02 22:23:24.999, 2022-12-02 22:23:23.997] -[2022-12-02 22:23:24.999, 2022-12-02 23:23:24.999, 2022-12-02 22:23:24.999, 2022-12-02 22:23:23.997] +[2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000, 2022-12-02 22:23:24.999, 2022-12-02 22:23:23.997] +[2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000, 2022-12-02 22:23:24.999, 2022-12-02 22:23:23.997] -- !select -- [22.679, 33.679, 22.679, 33.679, 22.679, 33.679] [23.679, 34.679, 23.679, 34.679, 23.679, 34.679] -- !select -- -[{22.679, 22.679, 2022-12-01 22:23:24.999, 22.679}, {33.679, 33.679, 2022-12-01 23:23:24.999, 33.679}] -[{23.679, 23.679, 2022-12-02 22:23:24.999, 23.679}, {34.679, 34.679, 2022-12-02 23:23:24.999, 34.679}] +[{22.679, 22.679, 2022-12-01 22:23:25.000, 22.679}, {33.679, 33.679, 2022-12-01 23:23:25.000, 33.679}] +[{23.679, 23.679, 2022-12-02 22:23:25.000, 23.679}, {34.679, 34.679, 2022-12-02 23:23:25.000, 34.679}] -- !select -- [{2022-12-02 22:23:24.999}, {2022-12-02 22:23:23.997}] [{2022-12-02 22:23:24.999}, {2022-12-02 22:23:23.997}] -- !select -- -[{2022-12-01 22:23:24.999}, {2022-12-01 23:23:24.999}] -[{2022-12-02 22:23:24.999}, {2022-12-02 23:23:24.999}] +[{2022-12-01 22:23:25.000}, {2022-12-01 23:23:25.000}] +[{2022-12-02 22:23:25.000}, {2022-12-02 23:23:25.000}] -- !select -- -[{2022-12-01 22:23:24.999, 2022-12-02 22:23:24.999}, {2022-12-01 23:23:24.999, 2022-12-02 22:23:23.997}] -[{2022-12-02 22:23:24.999, 2022-12-02 22:23:24.999}, {2022-12-02 23:23:24.999, 2022-12-02 22:23:23.997}] +[{2022-12-01 22:23:25.000, 2022-12-02 22:23:24.999}, {2022-12-01 23:23:25.000, 2022-12-02 22:23:23.997}] +[{2022-12-02 22:23:25.000, 2022-12-02 22:23:24.999}, {2022-12-02 23:23:25.000, 2022-12-02 22:23:23.997}] -- !select -- [2022-12-02 22:23:23.997, 2022-12-02 22:23:24.999] [2022-12-02 22:23:23.997, 2022-12-02 22:23:24.999] -- !select -- -[2023-03-08 23:23:23.997, 2022-12-01 22:23:24.999, 2022-12-01 23:23:24.999] -[2023-03-08 23:23:23.997, 2022-12-02 22:23:24.999, 2022-12-02 23:23:24.999] +[2023-03-08 23:23:23.997, 2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000] +[2023-03-08 23:23:23.997, 2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000] -- !select -- -2022-12-01T22:23:24.999 [2022-12-01 22:23:24.999, 2022-12-01 23:23:24.999] [2022-12-01 22:23:24.999, 2022-12-01 22:23:24.999, 2022-12-01 23:23:24.999] -2022-12-02T22:23:24.999 [2022-12-02 22:23:24.999, 2022-12-02 23:23:24.999] [2022-12-02 22:23:24.999, 2022-12-02 22:23:24.999, 2022-12-02 23:23:24.999] +2022-12-01T22:23:25 [2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000] [2022-12-01 22:23:25.000, 2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000] +2022-12-02T22:23:25 [2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000] [2022-12-02 22:23:25.000, 2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000] -- !select -- [25.990, 22.679, 33.679] @@ -167,12 +167,12 @@ [2022-12-02 22:23:24.999, 2022-12-02 22:23:23.997] -- !select -- -[2022-12-01 22:23:24.999, 2022-12-01 23:23:24.999, 2023-03-08 23:23:23.997] -[2022-12-02 22:23:24.999, 2022-12-02 23:23:24.999, 2023-03-08 23:23:23.997] +[2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000, 2023-03-08 23:23:23.997] +[2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000, 2023-03-08 23:23:23.997] -- !select -- -2022-12-01T22:23:24.999 [2022-12-01 22:23:24.999, 2022-12-01 23:23:24.999] [2022-12-01 22:23:24.999, 2022-12-01 23:23:24.999, 2022-12-01 22:23:24.999] -2022-12-02T22:23:24.999 [2022-12-02 22:23:24.999, 2022-12-02 23:23:24.999] [2022-12-02 22:23:24.999, 2022-12-02 23:23:24.999, 2022-12-02 22:23:24.999] +2022-12-01T22:23:25 [2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000] [2022-12-01 22:23:25.000, 2022-12-01 23:23:25.000, 2022-12-01 22:23:25.000] +2022-12-02T22:23:25 [2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000] [2022-12-02 22:23:25.000, 2022-12-02 23:23:25.000, 2022-12-02 22:23:25.000] -- !select -- [22.679, 33.679, 25.990] diff --git a/regression-test/data/query_p0/sql_functions/cast_function/test_cast_with_scale_type.out b/regression-test/data/query_p0/sql_functions/cast_function/test_cast_with_scale_type.out index 876b45e501a558..2fed8d279e4243 100644 --- a/regression-test/data/query_p0/sql_functions/cast_function/test_cast_with_scale_type.out +++ b/regression-test/data/query_p0/sql_functions/cast_function/test_cast_with_scale_type.out @@ -1,11 +1,11 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select1 -- -1 2022-12-01T22:23:24.999 2022-12-01 22:23:24.999999 -2 2022-12-02T22:23:24.999 2022-12-02 22:23:24.999999 +1 2022-12-01T22:23:25 2022-12-01 22:23:24.999999 +2 2022-12-02T22:23:25 2022-12-02 22:23:24.999999 -- !select2 -- -1 2022-12-01T22:23:24.999 2022-12-01T22:23:25 -2 2022-12-02T22:23:24.999 2022-12-02T22:23:25 +1 2022-12-01T22:23:25 2022-12-01T22:23:25 +2 2022-12-02T22:23:25 2022-12-02T22:23:25 -- !select3 -- 2022-12-02T22:23:25 2022-12-02T22:23:24 diff --git a/regression-test/data/query_p0/sql_functions/struct_functions/test_struct_functions.out b/regression-test/data/query_p0/sql_functions/struct_functions/test_struct_functions.out index ffce5c3e1968eb..2d09d7a2af4d57 100644 Binary files a/regression-test/data/query_p0/sql_functions/struct_functions/test_struct_functions.out and b/regression-test/data/query_p0/sql_functions/struct_functions/test_struct_functions.out differ diff --git a/regression-test/data/query_p0/union/test_union_pipelineX.out b/regression-test/data/query_p0/union/test_union_pipelineX.out deleted file mode 100644 index 0be613ddb45b3a..00000000000000 --- a/regression-test/data/query_p0/union/test_union_pipelineX.out +++ /dev/null @@ -1,369 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !union1 -- -123.123 0.1 true - --- !union2 -- -123.123 0.1 true world - --- !union3 -- - --- !union4 -- --654.654 0.0 --0.123 987456.123 -0.000 3.141592653 -0.666 -987.001 -123.123 0.1 -243.325 -0.0 -604587.000 0.1 - --- !union5 -- -false 1 1989 1001 11011902 123.123 true 1989-03-21 1989-03-21T13:00 wangjuoo4 0.1 6.333 string12345 170141183460469231731687303715884105727 -false 2 1986 1001 11011903 1243.500 false 1901-12-31 1989-03-21T13:00 wangynnsf 20.268 789.25 string12345 -170141183460469231731687303715884105727 -false 3 1989 1002 11011905 24453.325 false 2012-03-14 2000-01-01T00:00 yunlj8@nk 78945.0 3654.0 string12345 0 - --- !union6 -- -false 1 1989 1001 11011902 123.123 true 1989-03-21 1989-03-21T13:00 wangjuoo4 0.1 6.333 string12345 170141183460469231731687303715884105727 -false 1 1989 1001 11011902 123.123 true 1989-03-21 1989-03-21T13:00 wangjuoo4 0.1 6.333 string12345 170141183460469231731687303715884105727 -false 2 1986 1001 11011903 1243.500 false 1901-12-31 1989-03-21T13:00 wangynnsf 20.268 789.25 string12345 -170141183460469231731687303715884105727 -false 2 1986 1001 11011903 1243.500 false 1901-12-31 1989-03-21T13:00 wangynnsf 20.268 789.25 string12345 -170141183460469231731687303715884105727 - --- !union7 -- -false 1 1989 1001 11011902 123.123 true 1989-03-21 1989-03-21T13:00 wangjuoo4 0.1 6.333 string12345 170141183460469231731687303715884105727 -false 2 1986 1001 11011903 1243.500 false 1901-12-31 1989-03-21T13:00 wangynnsf 20.268 789.25 string12345 -170141183460469231731687303715884105727 -false 3 1989 1002 11011905 24453.325 false 2012-03-14 2000-01-01T00:00 yunlj8@nk 78945.0 3654.0 string12345 0 - --- !union8 -- -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -2 1986 1001 11011903 1243.500 false wangynnsf 20.268 789.25 1901-12-31 1989-03-21T13:00 -2 1986 1001 11011903 1243.500 false wangynnsf 20.268 789.25 1901-12-31 1989-03-21T13:00 -2 1986 1001 11011903 1243.500 false wangynnsf 20.268 789.25 1901-12-31 1989-03-21T13:00 -3 1989 1002 11011905 24453.325 false yunlj8@nk 78945.0 3654.0 2012-03-14 2000-01-01T00:00 -3 1989 1002 11011905 24453.325 false yunlj8@nk 78945.0 3654.0 2012-03-14 2000-01-01T00:00 -3 1989 1002 11011905 24453.325 false yunlj8@nk 78945.0 3654.0 2012-03-14 2000-01-01T00:00 -4 1991 3021 -11011907 243243.325 false yanvjldjlll 2.06 -0.001 3124-10-10 2015-03-13T10:30 -4 1991 3021 -11011907 243243.325 false yanvjldjlll 2.06 -0.001 3124-10-10 2015-03-13T10:30 -5 1985 5014 -11011903 243.325 true du3lnvl -0.0 -365.0 2015-01-01 2015-03-13T12:36:38 -5 1985 5014 -11011903 243.325 true du3lnvl -0.0 -365.0 2015-01-01 2015-03-13T12:36:38 -6 32767 3021 123456 604587.000 true yanavnd 0.1 80699.0 2014-11-11 2015-03-13T12:36:38 -6 32767 3021 123456 604587.000 true yanavnd 0.1 80699.0 2014-11-11 2015-03-13T12:36:38 -7 -32767 1002 7210457 3.141 false jiw3n4 0.0 6058.0 1988-03-21 1901-01-01T00:00 -8 255 2147483647 11011920 -0.123 true wangjuoo5 987456.123 12.14 1989-03-21 9999-11-11T12:12 -8 255 2147483647 11011920 -0.123 true wangjuoo5 987456.123 12.14 1989-03-21 9999-11-11T12:12 -9 1991 -2147483647 11011902 -654.654 true wangjuoo4 0.0 69.123 1991-08-11 1989-03-21T13:11 -10 1991 5014 9223372036854775807 -258.369 false wangynnsf -123456.54 0.235 2015-04-02 2013-04-02T15:16:52 -10 1991 5014 9223372036854775807 -258.369 false wangynnsf -123456.54 0.235 2015-04-02 2013-04-02T15:16:52 -11 1989 25699 -9223372036854775807 0.666 true yunlj8@nk -987.001 4.336 2015-04-02 1989-03-21T13:11 -11 1989 25699 -9223372036854775807 0.666 true yunlj8@nk -987.001 4.336 2015-04-02 1989-03-21T13:11 -12 32767 -2147483647 9223372036854775807 243.325 false lifsno -564.898 3.1415927 1991-08-11 2013-04-02T15:16:52 -13 -32767 2147483647 -9223372036854775807 100.001 false wenlsfnl 123.456 3.1415927 2015-04-02 2015-04-02T00:00 -14 255 103 11011902 0.000 false 3.141592654 2.036 2015-04-02 2015-04-02T00:00 -14 255 103 11011902 0.000 false 3.141592654 2.036 2015-04-02 2015-04-02T00:00 -15 1992 3021 11011920 0.000 true 3.141592653 20.456 9999-12-12 2015-04-02T00:00 -15 1992 3021 11011920 0.000 true 3.141592653 20.456 9999-12-12 2015-04-02T00:00 - --- !union9 -- -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 - --- !union10 -- -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -2 1986 1001 11011903 1243.500 false wangynnsf 20.268 789.25 1901-12-31 1989-03-21T13:00 -3 1989 1002 11011905 24453.325 false yunlj8@nk 78945.0 3654.0 2012-03-14 2000-01-01T00:00 -4 1991 3021 -11011907 243243.325 false yanvjldjlll 2.06 -0.001 3124-10-10 2015-03-13T10:30 -5 1985 5014 -11011903 243.325 true du3lnvl -0.0 -365.0 2015-01-01 2015-03-13T12:36:38 -6 32767 3021 123456 604587.000 true yanavnd 0.1 80699.0 2014-11-11 2015-03-13T12:36:38 -7 -32767 1002 7210457 3.141 false jiw3n4 0.0 6058.0 1988-03-21 1901-01-01T00:00 -8 255 2147483647 11011920 -0.123 true wangjuoo5 987456.123 12.14 1989-03-21 9999-11-11T12:12 -9 1991 -2147483647 11011902 -654.654 true wangjuoo4 0.0 69.123 1991-08-11 1989-03-21T13:11 -10 1991 5014 9223372036854775807 -258.369 false wangynnsf -123456.54 0.235 2015-04-02 2013-04-02T15:16:52 -11 1989 25699 -9223372036854775807 0.666 true yunlj8@nk -987.001 4.336 2015-04-02 1989-03-21T13:11 -12 32767 -2147483647 9223372036854775807 243.325 false lifsno -564.898 3.1415927 1991-08-11 2013-04-02T15:16:52 -13 -32767 2147483647 -9223372036854775807 100.001 false wenlsfnl 123.456 3.1415927 2015-04-02 2015-04-02T00:00 -14 255 103 11011902 0.000 false 3.141592654 2.036 2015-04-02 2015-04-02T00:00 -15 1992 3021 11011920 0.000 true 3.141592653 20.456 9999-12-12 2015-04-02T00:00 - --- !union11 -- -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -2 1986 1001 11011903 1243.500 false wangynnsf 20.268 789.25 1901-12-31 1989-03-21T13:00 -2 1986 1001 11011903 1243.500 false wangynnsf 20.268 789.25 1901-12-31 1989-03-21T13:00 -3 1989 1002 11011905 24453.325 false yunlj8@nk 78945.0 3654.0 2012-03-14 2000-01-01T00:00 -3 1989 1002 11011905 24453.325 false yunlj8@nk 78945.0 3654.0 2012-03-14 2000-01-01T00:00 -4 1991 3021 -11011907 243243.325 false yanvjldjlll 2.06 -0.001 3124-10-10 2015-03-13T10:30 -4 1991 3021 -11011907 243243.325 false yanvjldjlll 2.06 -0.001 3124-10-10 2015-03-13T10:30 -5 1985 5014 -11011903 243.325 true du3lnvl -0.0 -365.0 2015-01-01 2015-03-13T12:36:38 -5 1985 5014 -11011903 243.325 true du3lnvl -0.0 -365.0 2015-01-01 2015-03-13T12:36:38 -6 32767 3021 123456 604587.000 true yanavnd 0.1 80699.0 2014-11-11 2015-03-13T12:36:38 -6 32767 3021 123456 604587.000 true yanavnd 0.1 80699.0 2014-11-11 2015-03-13T12:36:38 -7 -32767 1002 7210457 3.141 false jiw3n4 0.0 6058.0 1988-03-21 1901-01-01T00:00 -8 255 2147483647 11011920 -0.123 true wangjuoo5 987456.123 12.14 1989-03-21 9999-11-11T12:12 -8 255 2147483647 11011920 -0.123 true wangjuoo5 987456.123 12.14 1989-03-21 9999-11-11T12:12 -9 1991 -2147483647 11011902 -654.654 true wangjuoo4 0.0 69.123 1991-08-11 1989-03-21T13:11 -10 1991 5014 9223372036854775807 -258.369 false wangynnsf -123456.54 0.235 2015-04-02 2013-04-02T15:16:52 -10 1991 5014 9223372036854775807 -258.369 false wangynnsf -123456.54 0.235 2015-04-02 2013-04-02T15:16:52 -11 1989 25699 -9223372036854775807 0.666 true yunlj8@nk -987.001 4.336 2015-04-02 1989-03-21T13:11 -11 1989 25699 -9223372036854775807 0.666 true yunlj8@nk -987.001 4.336 2015-04-02 1989-03-21T13:11 -12 32767 -2147483647 9223372036854775807 243.325 false lifsno -564.898 3.1415927 1991-08-11 2013-04-02T15:16:52 -13 -32767 2147483647 -9223372036854775807 100.001 false wenlsfnl 123.456 3.1415927 2015-04-02 2015-04-02T00:00 -14 255 103 11011902 0.000 false 3.141592654 2.036 2015-04-02 2015-04-02T00:00 -14 255 103 11011902 0.000 false 3.141592654 2.036 2015-04-02 2015-04-02T00:00 -15 1992 3021 11011920 0.000 true 3.141592653 20.456 9999-12-12 2015-04-02T00:00 -15 1992 3021 11011920 0.000 true 3.141592653 20.456 9999-12-12 2015-04-02T00:00 - --- !union12 -- -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 - --- !union14 -- -1 2 -3 4 -10 20 - --- !union15 -- -6 - --- !union16 -- -1 1 -2 3 - --- !union17 -- -1 a \N 10.0 -1 a \N 10.0 -2 b \N 20.0 - --- !union18 -- -16 - --- !union19 -- -1 123.123 true wangjuoo4 -10 10.0 hello world -20 20.0 wangjuoo4 beautiful -255 3.141592654 false -255 987456.123 true wangjuoo5 -1985 -0.0 true du3lnvl -1986 20.268 false wangynnsf -1989 -987.001 true yunlj8@nk -1989 0.1 true wangjuoo4 -1989 78945.0 false yunlj8@nk -1991 -123456.54 false wangynnsf -1991 0.0 true wangjuoo4 -1991 2.06 false yanvjldjlll -1992 3.141592653 true -32767 -564.898 false lifsno -32767 0.1 true yanavnd - --- !union20 -- -1 1989 1001 11011902 123.123 -2 1986 1001 11011903 1243.500 -3 1989 1002 11011905 24453.325 -4 1991 3021 -11011907 243243.325 - --- !union21 -- -1 1989 1001 11011902 123.123 -1 1989 1001 11011902 123.123 -2 1986 1001 11011903 1243.500 -2 1986 1001 11011903 1243.500 -3 1989 1002 11011905 24453.325 -3 1989 1002 11011905 24453.325 -4 1991 3021 -11011907 243243.325 - --- !union22 -- -\N \N \N \N \N \N -1 true wangjuoo4 0.1 6.333 1989-03-21 -2 false wangynnsf 20.268 789.25 1901-12-31 -3 false yunlj8@nk 78945.0 3654.0 2012-03-14 -4 false yanvjldjlll 2.06 -0.001 3124-10-10 -5 true du3lnvl -0.0 -365.0 2015-01-01 -6 true yanavnd 0.1 80699.0 2014-11-11 -7 false jiw3n4 0.0 6058.0 1988-03-21 -8 true wangjuoo5 987456.123 12.14 1989-03-21 -9 true wangjuoo4 0.0 69.123 1991-08-11 - --- !union23 -- -\N \N \N \N \N \N -1 true wangjuoo4 0.1 6.333 1989-03-21 -1 true wangjuoo4 0.1 6.333 1989-03-21 -1 true wangjuoo4 0.1 6.333 1989-03-21 -2 false wangynnsf 20.268 789.25 1901-12-31 -2 false wangynnsf 20.268 789.25 1901-12-31 -3 false yunlj8@nk 78945.0 3654.0 2012-03-14 -3 false yunlj8@nk 78945.0 3654.0 2012-03-14 -4 false yanvjldjlll 2.06 -0.001 3124-10-10 -5 true du3lnvl -0.0 -365.0 2015-01-01 - --- !union24 -- -1 2 3 4 3.140 hello world 0.0 1.1 1989-03-21 1989-03-21T13:00 -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -2 1986 1001 11011903 1243.500 false wangynnsf 20.268 789.25 1901-12-31 1989-03-21T13:00 -2 1986 1001 11011903 1243.500 false wangynnsf 20.268 789.25 1901-12-31 1989-03-21T13:00 -3 1989 1002 11011905 24453.325 false yunlj8@nk 78945.0 3654.0 2012-03-14 2000-01-01T00:00 -3 1989 1002 11011905 24453.325 false yunlj8@nk 78945.0 3654.0 2012-03-14 2000-01-01T00:00 -4 1991 3021 -11011907 243243.325 false yanvjldjlll 2.06 -0.001 3124-10-10 2015-03-13T10:30 -5 1985 5014 -11011903 243.325 true du3lnvl -0.0 -365.0 2015-01-01 2015-03-13T12:36:38 -6 32767 3021 123456 604587.000 true yanavnd 0.1 80699.0 2014-11-11 2015-03-13T12:36:38 -7 -32767 1002 7210457 3.141 false jiw3n4 0.0 6058.0 1988-03-21 1901-01-01T00:00 -8 255 2147483647 11011920 -0.123 true wangjuoo5 987456.123 12.14 1989-03-21 9999-11-11T12:12 -10 1991 5014 9223372036854775807 -258.369 false wangynnsf -123456.54 0.235 2015-04-02 2013-04-02T15:16:52 -11 1989 25699 -9223372036854775807 0.666 true yunlj8@nk -987.001 4.336 2015-04-02 1989-03-21T13:11 -13 -32767 2147483647 -9223372036854775807 100.001 false wenlsfnl 123.456 3.1415927 2015-04-02 2015-04-02T00:00 -14 255 103 11011902 0.000 false 3.141592654 2.036 2015-04-02 2015-04-02T00:00 -15 1992 3021 11011920 0.000 true 3.141592653 20.456 9999-12-12 2015-04-02T00:00 - --- !union25 -- -1 2 3 4 3.140 hello world 0.0 1.1 1989-03-21 1989-03-21T13:00 -1 1989 1001 11011902 123.123 true wangjuoo4 0.1 6.333 1989-03-21 1989-03-21T13:00 -2 1986 1001 11011903 1243.500 false wangynnsf 20.268 789.25 1901-12-31 1989-03-21T13:00 -3 1989 1002 11011905 24453.325 false yunlj8@nk 78945.0 3654.0 2012-03-14 2000-01-01T00:00 -4 1991 3021 -11011907 243243.325 false yanvjldjlll 2.06 -0.001 3124-10-10 2015-03-13T10:30 -5 1985 5014 -11011903 243.325 true du3lnvl -0.0 -365.0 2015-01-01 2015-03-13T12:36:38 -6 32767 3021 123456 604587.000 true yanavnd 0.1 80699.0 2014-11-11 2015-03-13T12:36:38 -7 -32767 1002 7210457 3.141 false jiw3n4 0.0 6058.0 1988-03-21 1901-01-01T00:00 -8 255 2147483647 11011920 -0.123 true wangjuoo5 987456.123 12.14 1989-03-21 9999-11-11T12:12 -10 1991 5014 9223372036854775807 -258.369 false wangynnsf -123456.54 0.235 2015-04-02 2013-04-02T15:16:52 -11 1989 25699 -9223372036854775807 0.666 true yunlj8@nk -987.001 4.336 2015-04-02 1989-03-21T13:11 -13 -32767 2147483647 -9223372036854775807 100.001 false wenlsfnl 123.456 3.1415927 2015-04-02 2015-04-02T00:00 -14 255 103 11011902 0.000 false 3.141592654 2.036 2015-04-02 2015-04-02T00:00 -15 1992 3021 11011920 0.000 true 3.141592653 20.456 9999-12-12 2015-04-02T00:00 - --- !union26 -- -0.0001 1E-7 -1.0000 2.0000000 -1.0100 2.0000000 - --- !union27 -- -1 2 -hell0 - --- !union28 -- -1.00000000 2.00000 - --- !union29 -- -1.00000000 2.00000 - --- !union30 -- -1.00000000 2.00000 -1.00000000 2.00000 -1.00000000 2.00000 - --- !union31 -- -1.00000000 2.00000 -1.00000000 2.00000 - --- !union32 -- -1.00000000 2.00000 - --- !union33 -- -2016-07-01 -2016-07-02 - --- !union34 -- -2016-07-01 -2016-07-02 - --- !union35 -- -2016-07-01 -2016-07-02 - --- !union36 -- -1 2 - --- !union37 -- -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -1 -10 -2 -3 -4 -5 -6 -7 -8 -9 - --- !union38 -- -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -1 -2 -3 -4 -5 -6 -7 -8 -9 -10 - --- !union39 -- -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -1 -2 -3 -4 -5 -6 -7 -8 -9 -10 - --- !union40 -- -1 -2 -3 - --- !union40 -- -1986 -1989 - --- !union40 -- -1001 -1002 - --- !union40 -- -11011902 -11011903 -11011905 - --- !union35 -- -2016-07-01 -2016-07-02 - --- !union36 -- -1 2 - diff --git a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.out b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.out index 0f1cd09e7dc617..eb0a501090a687 100644 --- a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.out +++ b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.out @@ -52,3 +52,19 @@ 4 4 4 4 4 0 5 \N \N \N \N 1 +-- !1 -- +1 1 1 + +-- !2 -- + +-- !3 -- +1 2 \N + +-- !1 -- +1 1 1 1 + +-- !2 -- + +-- !3 -- +1 2 \N \N + diff --git a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign_data.csv b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign_data.csv new file mode 100644 index 00000000000000..d72f2010a82217 --- /dev/null +++ b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete_sign_data.csv @@ -0,0 +1 @@ +1,2 diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy index 2dc4a0c8fab50e..0fe263f2919b70 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy @@ -139,6 +139,10 @@ class StreamLoadAction implements SuiteAction { headers.put(key, value) } + void unset(String key) { + headers.remove(key) + } + @Override void run() { String responseText = null diff --git a/regression-test/suites/alter_p2/test_alter_colocate_group.groovy b/regression-test/suites/alter_p2/test_alter_colocate_group.groovy new file mode 100644 index 00000000000000..1f5b8496630b40 --- /dev/null +++ b/regression-test/suites/alter_p2/test_alter_colocate_group.groovy @@ -0,0 +1,170 @@ +// 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_alter_colocate_group") { + sql "DROP DATABASE IF EXISTS test_alter_colocate_group_db FORCE" + test { + sql """ + ALTER COLOCATE GROUP test_alter_colocate_group_db.bad_group_1 + SET ( "replication_num" = "1" ); + """ + + exception "unknown databases" + } + test { + sql """ + ALTER COLOCATE GROUP bad_group_2 + SET ( "replication_num" = "1" ); + """ + + exception "Not found colocate group `default_cluster:regression_test_alter_p2`.`bad_group_2`" + } + test { + sql """ + ALTER COLOCATE GROUP bad_db.__global__bad_group_3 + SET ( "replication_num" = "1" ); + """ + + exception "group that name starts with `__global__` is a global group, it doesn't belong to any specific database" + } + test { + sql """ + ALTER COLOCATE GROUP __global__bad_group_4 + SET ( "replication_num" = "1" ); + """ + + exception "Not found colocate group `__global__bad_group_4`" + } + + sql " DROP TABLE IF EXISTS tbl1 FORCE; " + sql " DROP TABLE IF EXISTS tbl2 FORCE; " + sql " DROP TABLE IF EXISTS tbl3 FORCE; " + + sql """ + CREATE TABLE tbl1 + ( + k1 int, + k2 int + ) + DISTRIBUTED BY HASH(k1) BUCKETS 6 + PROPERTIES + ( + "colocate_with" = "group_1", + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE tbl2 + ( + k1 date, + k2 int + ) + PARTITION BY RANGE(k1) + ( + PARTITION p1 values less than('2020-02-01'), + PARTITION p2 values less than('2020-03-01') + ) + DISTRIBUTED BY HASH(k2) BUCKETS 5 + PROPERTIES + ( + "colocate_with" = "group_2", + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE tbl3 + ( + `uuid` varchar(255) NULL, + `action_datetime` date NULL + ) + DUPLICATE KEY(uuid) + PARTITION BY RANGE(action_datetime)() + DISTRIBUTED BY HASH(uuid) BUCKETS 4 + PROPERTIES + ( + "colocate_with" = "group_3", + "replication_num" = "1", + "dynamic_partition.enable" = "true", + "dynamic_partition.time_unit" = "DAY", + "dynamic_partition.end" = "2", + "dynamic_partition.prefix" = "p", + "dynamic_partition.buckets" = "4", + "dynamic_partition.replication_num" = "1" + ); + """ + + def checkGroupsReplicaAlloc = { groupName, replicaNum -> + // groupName -> replicaAlloc + def allocMap = [:] + def groups = sql """ show proc "/colocation_group" """ + for (def group : groups) { + allocMap[group[1]] = group[4] + } + + assertEquals("tag.location.default: ${replicaNum}".toString(), allocMap[groupName]) + } + + def checkTableReplicaAlloc = { tableName, hasDynamicPart, replicaNum -> + def result = sql """ show create table ${tableName} """ + def createTbl = result[0][1].toString() + assertTrue(createTbl.indexOf("\"replication_allocation\" = \"tag.location.default: ${replicaNum}\"") > 0) + if (hasDynamicPart) { + assertTrue(createTbl.indexOf( + "\"dynamic_partition.replication_allocation\" = \"tag.location.default: ${replicaNum}\"") > 0) + } + + result = sql """ show partitions from ${tableName} """ + assertTrue(result.size() > 0) + for (int i = 0; i < result.size(); i++) { + assertEquals("${replicaNum}".toString(), result[i][9].toString()) + } + } + + for (int i = 1; i <= 3; i++) { + def groupName = "regression_test_alter_p2.group_${i}" + checkGroupsReplicaAlloc(groupName, 1) + + def tableName = "tbl${i}" + def hasDynamicPart = i == 3 + checkTableReplicaAlloc(tableName, hasDynamicPart, 1) + + test { + sql """ + ALTER COLOCATE GROUP ${groupName} + SET ( "replication_num" = "100" ); + """ + + exception "Failed to find enough host" + } + + test { + sql """ + ALTER COLOCATE GROUP ${groupName} + SET ( "replication_num" = "3" ); + """ + } + + checkGroupsReplicaAlloc(groupName, 3) + checkTableReplicaAlloc(tableName, hasDynamicPart, 3) + } + + sql " DROP TABLE IF EXISTS tbl1 FORCE; " + sql " DROP TABLE IF EXISTS tbl2 FORCE; " + sql " DROP TABLE IF EXISTS tbl3 FORCE; " +} diff --git a/regression-test/suites/autobucket/test_autobucket.groovy b/regression-test/suites/autobucket/test_autobucket.groovy index ab0ae99658b57b..d3ba70d0df35a9 100644 --- a/regression-test/suites/autobucket/test_autobucket.groovy +++ b/regression-test/suites/autobucket/test_autobucket.groovy @@ -39,4 +39,27 @@ suite("test_autobucket") { assertEquals(Integer.valueOf(result.get(0).get(8)), 10) sql "drop table if exists autobucket_test" + + + sql "drop table if exists autobucket_test_min_buckets" + result = sql """ + CREATE TABLE `autobucket_test_min_buckets` ( + `user_id` largeint(40) NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`user_id`) BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "estimate_partition_size" = "1M" + ) + """ + + default_min_buckets = 1 // in Config.java + result = sql "show partitions from autobucket_test_min_buckets" + logger.info("${result}") + // XXX: buckets at pos(8), next maybe impl by sql meta + assertEquals(Integer.valueOf(result.get(0).get(8)), default_min_buckets) + + sql "drop table if exists autobucket_test_min_buckets" } diff --git a/regression-test/suites/correctness/test_time_function.groovy b/regression-test/suites/correctness/test_time_function.groovy index c8e88af585b368..c008d346da4436 100644 --- a/regression-test/suites/correctness/test_time_function.groovy +++ b/regression-test/suites/correctness/test_time_function.groovy @@ -19,24 +19,115 @@ suite("test_time_function") { sql """ set enable_nereids_planner=true,enable_fallback_to_original_planner=false """ - qt_select1 """ + qt_select """ select sec_to_time(time_to_sec(cast('16:32:18' as time))); """ - qt_select2 """ + qt_select """ select sec_to_time(59538); """ + // "HHMMSS" + qt_select """ + select cast("123456" as TIME); + """ + qt_select """ + select cast("3456" as TIME); + """ + qt_select """ + select cast("56" as TIME); + """ + + // "HH:MM:SS" + qt_select """ + select cast("12:34:56" as TIME); + """ + qt_select """ + select cast("34:56" as TIME); + """ + qt_select """ + select cast(":56" as TIME); + """ + + // HHMMSS + qt_select """ + select cast(123456 as TIME); + """ + qt_select """ + select cast(3456 as TIME); + """ + qt_select """ + select cast(56 as TIME); + """ + + // Invalid value in seconds part. + qt_select """ + select cast(":61" as TIME); + """ + qt_select """ + select cast("61" as TIME); + """ + qt_select """ + select cast(61 as TIME); + """ + + qt_select """ + select sec_to_time(time_to_sec(cast("61" as time))); + """ sql """ set enable_nereids_planner=false """ - qt_select3 """ + qt_select """ select sec_to_time(time_to_sec(cast('16:32:18' as time))); """ - qt_select4 """ + qt_select """ select sec_to_time(59538); """ + // "HHMMSS" + qt_select """ + select cast("123456" as TIME); + """ + qt_select """ + select cast("3456" as TIME); + """ + qt_select """ + select cast("56" as TIME); + """ + + // "HH:MM:SS" + qt_select """ + select cast("12:34:56" as TIME); + """ + qt_select """ + select cast("34:56" as TIME); + """ + qt_select """ + select cast(":56" as TIME); + """ + + // HHMMSS + qt_select """ + select cast(123456 as TIME); + """ + qt_select """ + select cast(3456 as TIME); + """ + qt_select """ + select cast(56 as TIME); + """ + // Invalid value in seconds part. + qt_select """ + select cast(":61" as TIME); + """ + qt_select """ + select cast("61" as TIME); + """ + qt_select """ + select cast(61 as TIME); + """ - + qt_select """ + select sec_to_time(time_to_sec(cast("61" as time))); + """ } \ No newline at end of file diff --git a/regression-test/suites/correctness_p0/test_inlineview_with_lateralview.groovy b/regression-test/suites/correctness_p0/test_inlineview_with_lateralview.groovy new file mode 100644 index 00000000000000..2559e7423d542d --- /dev/null +++ b/regression-test/suites/correctness_p0/test_inlineview_with_lateralview.groovy @@ -0,0 +1,58 @@ +// 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_inlineview_with_lateralview") { + sql "set enable_nereids_planner=false" + sql """ + drop table if exists lateralview_t1; + """ + + sql """ + create table lateralview_t1 ( + ip varchar(96) + ) + DISTRIBUTED by random BUCKETS 1 + PROPERTIES( + "replication_num" = "1" + ); + """ + sql """insert into lateralview_t1 values ('1');""" + + qt_select """SELECT max(position) + OVER (partition by idf) c_tk + FROM + (SELECT position, + idf + FROM + (SELECT lag(position, + 1, + 0) + OVER (partition by position) last_position, position, idf, gap + FROM + (SELECT idf, + position + 1 position, + gap + FROM + (SELECT ip AS idf, + 120 AS gap + FROM lateralview_t1 ) pro_actions lateral view explode_numbers(5) exp_tp AS position) events) last_position_events ) ttt;""" + + sql """ + drop table if exists lateralview_t1; + """ + +} diff --git a/regression-test/suites/datatype_p0/agg_state/nereids/test_agg_state_nereids.groovy b/regression-test/suites/datatype_p0/agg_state/nereids/test_agg_state_nereids.groovy index b31070b01ca2f8..c7a0a6d748dab5 100644 --- a/regression-test/suites/datatype_p0/agg_state/nereids/test_agg_state_nereids.groovy +++ b/regression-test/suites/datatype_p0/agg_state/nereids/test_agg_state_nereids.groovy @@ -55,15 +55,19 @@ suite("test_agg_state_nereids") { properties("replication_num" = "1"); """ + sql 'set enable_fallback_to_original_planner=true' sql "insert into a_table select 1,max_by_state(1,3);" sql "insert into a_table select 1,max_by_state(2,2);" sql "insert into a_table select 1,max_by_state(3,1);" + sql 'set enable_fallback_to_original_planner=false' qt_length1 """select k1,length(k2) from a_table order by k1;""" qt_group1 """select k1,max_by_merge(k2) from a_table group by k1 order by k1;""" qt_merge1 """select max_by_merge(k2) from a_table;""" - + + sql 'set enable_fallback_to_original_planner=true' sql "insert into a_table select k1+1, max_by_state(k2,k1+10) from d_table;" + sql 'set enable_fallback_to_original_planner=false' qt_length2 """select k1,length(k2) from a_table order by k1;""" qt_group2 """select k1,max_by_merge(k2) from a_table group by k1 order by k1;""" diff --git a/regression-test/suites/insert_p0/test_array_insert_overflow.groovy b/regression-test/suites/insert_p0/test_array_insert_overflow.groovy index 62c11494cbfdc1..68fa156ab199ee 100644 --- a/regression-test/suites/insert_p0/test_array_insert_overflow.groovy +++ b/regression-test/suites/insert_p0/test_array_insert_overflow.groovy @@ -16,6 +16,10 @@ // under the License. suite("test_array_insert_overflow") { + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + def testTable = "test_array_insert_overflow" sql """ diff --git a/regression-test/suites/inverted_index_p0/test_tokenize.groovy b/regression-test/suites/inverted_index_p0/test_tokenize.groovy index 7780329da0bc6b..5b5c4f02a444fb 100644 --- a/regression-test/suites/inverted_index_p0/test_tokenize.groovy +++ b/regression-test/suites/inverted_index_p0/test_tokenize.groovy @@ -91,4 +91,6 @@ suite("test_tokenize"){ sql "INSERT INTO $indexTblName3 VALUES (1, '我来到北京清华大学'), (2, '我爱你中国'), (3, '人民可以得到更多实惠'), (4, '陕西省西安市高新区创业大厦A座,我的手机号码是12345678901,邮箱是12345678@qq.com,,ip是1.1.1.1,this information is created automatically.');" qt_sql "SELECT TOKENIZE(c, \"'parser'='chinese','parser_mode'='fine_grained'\") FROM $indexTblName3"; + + qt_tokenize_sql """SELECT TOKENIZE('GET /images/hm_bg.jpg HTTP/1.0 test:abc=bcd','"parser"="unicode","char_filter_type" = "char_replace","char_filter_pattern" = "._=:,","char_filter_replacement" = " "');""" } 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 6f728330722f38..0d8b9804464c19 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 @@ -19,6 +19,9 @@ import org.codehaus.groovy.runtime.IOGroovyMethods suite("test_json_load_and_function", "p0") { + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + // define a sql table def testTable = "tbl_test_json" def dataFile = "test_json.csv" diff --git a/regression-test/suites/json_p0/test_json_load_unique_key_and_function.groovy b/regression-test/suites/json_p0/test_json_load_unique_key_and_function.groovy index 4b990bc4f7935d..7d1dad49405121 100644 --- a/regression-test/suites/json_p0/test_json_load_unique_key_and_function.groovy +++ b/regression-test/suites/json_p0/test_json_load_unique_key_and_function.groovy @@ -16,6 +16,10 @@ // under the License. suite("test_json_unique_load_and_function", "p0") { + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + // define a sql table def testTable = "tbl_test_json_unique" def dataFile = "test_json_unique_key.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 46b64813e02c67..1347726d2afe06 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 @@ -18,6 +18,10 @@ import org.codehaus.groovy.runtime.IOGroovyMethods suite("test_jsonb_load_and_function", "p0") { + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + // define a sql table def testTable = "tbl_test_jsonb" def dataFile = "test_jsonb.csv" diff --git a/regression-test/suites/jsonb_p0/test_jsonb_load_unique_key_and_function.groovy b/regression-test/suites/jsonb_p0/test_jsonb_load_unique_key_and_function.groovy index 3f53721d4bff09..398e0f06eda803 100644 --- a/regression-test/suites/jsonb_p0/test_jsonb_load_unique_key_and_function.groovy +++ b/regression-test/suites/jsonb_p0/test_jsonb_load_unique_key_and_function.groovy @@ -16,6 +16,9 @@ // under the License. suite("test_jsonb_unique_load_and_function", "p0") { + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" // define a sql table def testTable = "tbl_test_jsonb_unique" def dataFile = "test_jsonb_unique_key.csv" diff --git a/regression-test/suites/load_p0/http_stream/test_group_commit_http_stream.groovy b/regression-test/suites/load_p0/http_stream/test_group_commit_http_stream.groovy new file mode 100644 index 00000000000000..a27963742dd8b0 --- /dev/null +++ b/regression-test/suites/load_p0/http_stream/test_group_commit_http_stream.groovy @@ -0,0 +1,328 @@ +// 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_group_commit_http_stream") { + def db = "regression_test_load_p0_http_stream" + def tableName = "test_group_commit_http_stream" + + def getRowCount = { expectedRowCount -> + def retry = 0 + while (retry < 10) { + sleep(2000) + def rowCount = sql "select count(*) from ${tableName}" + logger.info("rowCount: " + rowCount + ", retry: " + retry) + if (rowCount[0][0] >= expectedRowCount) { + break + } + retry++ + } + } + + def getAlterTableState = { + def retry = 0 + while (true) { + sleep(8000) + def state = sql "show alter table column where tablename = '${tableName}' order by CreateTime desc " + logger.info("alter table retry: ${retry}, state: ${state}") + if (state.size() > 0 && state[0][9] == "FINISHED") { + return true + } + retry++ + if (retry >= 40) { + return false + } + } + return false + } + + try { + // create table + sql """ drop table if exists ${tableName}; """ + + sql """ + CREATE TABLE `${tableName}` ( + `id` int(11) NOT NULL, + `name` varchar(1100) NULL, + `score` int(11) NULL default "-1" + ) ENGINE=OLAP + DUPLICATE KEY(`id`, `name`) + PARTITION BY RANGE (id) ( + PARTITION plessThan1 VALUES LESS THAN ("0"), + PARTITION plessThan2 VALUES LESS THAN ("100") + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + // stream load with compress file + String[] compressionTypes = new String[]{"gz", "bz2", /*"lzo",*/ "lz4"} //, "deflate"} + /*for (final def compressionType in compressionTypes) { + def fileName = "test_compress.csv." + compressionType + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName} select * from http_stream + ("format"="csv", "compress_type"="${compressionType}", "column_separator"=",") + """ + set 'group_commit', 'true' + file "${fileName}" + unset 'label' + + time 10000 // limit inflight 10s + } + }*/ + + // stream load with 2 columns + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName}(id, name) select c1, c2 from http_stream + ("format"="csv", "column_separator"=",") + """ + + set 'group_commit', 'true' + file "test_stream_load1.csv" + unset 'label' + + time 10000 // limit inflight 10s + } + + // stream load with different column order + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName}(score, id, name) select c1, c2, c3 from http_stream + ("format"="csv", "column_separator"="|") + """ + + set 'group_commit', 'true' + file "test_stream_load2.csv" + unset 'label' + + time 10000 // limit inflight 10s + } + + // stream load with where condition + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName}(id, name) select c1, c2 from http_stream + ("format"="csv", "column_separator"=",") where c1 > 5 + """ + + set 'group_commit', 'true' + file "test_stream_load1.csv" + unset 'label' + + time 10000 // limit inflight 10s + + 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()) + assertTrue(json.GroupCommit) + // assertEquals(2, json.NumberTotalRows) + assertEquals(1, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + // assertEquals(1, json.NumberUnselectedRows) + } + } + + // stream load with mapping + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName} select c1, c2, c1 * 10 from http_stream + ("format"="csv", "column_separator"=",") + """ + + set 'group_commit', 'true' + file "test_stream_load1.csv" + unset 'label' + + time 10000 // limit inflight 10s + } + + // stream load with filtered rows + /*streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName} select c1, c2, c3 from http_stream where c2 = 'a' + ("format"="csv", "column_separator"=",") + """ + + set 'group_commit', 'true' + file "test_stream_load3.csv" + set 'max_filter_ratio', '0.7' + unset 'label' + + time 10000 // limit inflight 10s + + 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()) + assertTrue(json.GroupCommit) + assertEquals(6, json.NumberTotalRows) + assertEquals(2, json.NumberLoadedRows) + assertEquals(3, json.NumberFilteredRows) + assertEquals(1, json.NumberUnselectedRows) + assertFalse(json.ErrorURL.isEmpty()) + } + }*/ + + // stream load with label + streamLoad { + set 'version', '1' + def label = 'l_' + System.currentTimeMillis() + set 'sql', """ + insert into ${db}.${tableName} with label ${label} select * from http_stream + ("format"="csv", "column_separator"="|") + """ + + set 'group_commit', 'true' + file "test_stream_load2.csv" + + time 10000 // limit inflight 10s + 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()) + } + } + + getRowCount(7) + qt_sql " SELECT * FROM ${tableName} order by id, name, score asc; " + } finally { + // try_sql("DROP TABLE ${tableName}") + } + + // stream load with large data and schema change + /*tableName = "test_stream_load_lineorder" + try { + sql """ DROP TABLE IF EXISTS `${tableName}` """ + sql """ + CREATE TABLE IF NOT EXISTS `${tableName}` ( + `lo_orderkey` bigint(20) NOT NULL COMMENT "", + `lo_linenumber` bigint(20) NOT NULL COMMENT "", + `lo_custkey` int(11) NOT NULL COMMENT "", + `lo_partkey` int(11) NOT NULL COMMENT "", + `lo_suppkey` int(11) NOT NULL COMMENT "", + `lo_orderdate` int(11) NOT NULL COMMENT "", + `lo_orderpriority` varchar(16) NOT NULL COMMENT "", + `lo_shippriority` int(11) NOT NULL COMMENT "", + `lo_quantity` bigint(20) NOT NULL COMMENT "", + `lo_extendedprice` bigint(20) NOT NULL COMMENT "", + `lo_ordtotalprice` bigint(20) NOT NULL COMMENT "", + `lo_discount` bigint(20) NOT NULL COMMENT "", + `lo_revenue` bigint(20) NOT NULL COMMENT "", + `lo_supplycost` bigint(20) NOT NULL COMMENT "", + `lo_tax` bigint(20) NOT NULL COMMENT "", + `lo_commitdate` bigint(20) NOT NULL COMMENT "", + `lo_shipmode` varchar(11) NOT NULL COMMENT "" + ) + PARTITION BY RANGE(`lo_orderdate`) + (PARTITION p1992 VALUES [("-2147483648"), ("19930101")), + PARTITION p1993 VALUES [("19930101"), ("19940101")), + PARTITION p1994 VALUES [("19940101"), ("19950101")), + PARTITION p1995 VALUES [("19950101"), ("19960101")), + PARTITION p1996 VALUES [("19960101"), ("19970101")), + PARTITION p1997 VALUES [("19970101"), ("19980101")), + PARTITION p1998 VALUES [("19980101"), ("19990101"))) + DISTRIBUTED BY HASH(`lo_orderkey`) BUCKETS 4 + PROPERTIES ( + "replication_num" = "1" + ); + """ + // load data + def columns = """lo_orderkey,lo_linenumber,lo_custkey,lo_partkey,lo_suppkey,lo_orderdate,lo_orderpriority, + lo_shippriority,lo_quantity,lo_extendedprice,lo_ordtotalprice,lo_discount, + lo_revenue,lo_supplycost,lo_tax,lo_commitdate,lo_shipmode""" + + new Thread(() -> { + Thread.sleep(3000) + // do light weight schema change + sql """ alter table ${tableName} ADD column sc_tmp varchar(100) after lo_revenue; """ + + assertTrue(getAlterTableState()) + + // do hard weight schema change + def new_columns = """lo_orderkey,lo_linenumber,lo_custkey,lo_partkey,lo_suppkey,lo_orderdate,lo_orderpriority, + lo_shippriority,lo_quantity,lo_extendedprice,lo_ordtotalprice,lo_discount, + lo_revenue,lo_supplycost,lo_tax,lo_shipmode,lo_commitdate""" + sql """ alter table ${tableName} order by (${new_columns}); """ + }).start(); + + for (int i = 0; i < 4; i++) { + + streamLoad { + set 'version', '1' + sql """ + insert into ${db}.${table} ($columns) + select c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17 from http_stream + ("format"="csv", "compress_type"="GZ", "column_separator"="|") + """ + table tableName + + // set 'column_separator', '|' + // set 'compress_type', 'GZ' + set 'columns', columns + ",lo_dummy" + set 'group_commit', 'true' + unset 'label' + + file """${getS3Url()}/regression/ssb/sf0.1/lineorder.tbl.gz""" + + time 10000 // limit inflight 10s + + // stream load action will check result, include Success status, and NumberTotalRows == NumberLoadedRows + + // 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 ${i}, result: ${result}") + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(json.NumberTotalRows, json.NumberLoadedRows) + assertEquals(json.NumberLoadedRows, 600572) + assertTrue(json.LoadBytes > 0) + assertTrue(json.GroupCommit) + } + } + } + + getRowCount(2402288) + qt_sql """ select count(*) from ${tableName} """ + + assertTrue(getAlterTableState()) + } finally { + // try_sql("DROP TABLE ${tableName}") + }*/ +} \ No newline at end of file diff --git a/regression-test/suites/load_p0/http_stream/test_http_stream_compress.groovy b/regression-test/suites/load_p0/http_stream/test_http_stream_compress.groovy new file mode 100644 index 00000000000000..17f580263dddb3 --- /dev/null +++ b/regression-test/suites/load_p0/http_stream/test_http_stream_compress.groovy @@ -0,0 +1,110 @@ +// 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_http_stream_compress", "p0") { + + // 1. test with gzip + def tableName1 = "test_http_stream_gzip" + def db = "regression_test_load_p0_http_stream" + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName1} ( + `k1` int(11) NULL, + `k2` tinyint(4) NULL, + `k3` smallint(6) NULL, + `k4` bigint(20) NULL, + `k5` largeint(40) NULL, + `k6` float NULL, + `k7` double NULL, + `k8` decimal(9, 0) NULL, + `k9` char(10) NULL, + `k10` varchar(1024) NULL, + `k11` text NULL, + `k12` date NULL, + `k13` datetime NULL + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`k1`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName1} select c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13 from http_stream("format"="csv", "compress_type"="GZ") + """ + time 10000 + file '../stream_load/all_types.csv.gz' + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("http_stream result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(2500, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql1 "select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11, k12, k13 from ${tableName1} order by k1" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName1}" + } + + // 2. test with bz2 + def tableName2 = "test_http_stream_bz4" + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName2} ( + `k1` int(11) NULL, + `k2` tinyint(4) NULL, + `v1` bitmap bitmap_union, + `v2` hll hll_union + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`k1`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName2} select c1, c2, to_bitmap(c3), hll_hash(c4) from http_stream("format"="csv", "compress_type"="bz2") + """ + time 10000 + file '../stream_load/bitmap_hll.csv.bz2' + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("http_stream result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(1025, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + sql "sync" + qt_sql2 "select k1, k2, bitmap_union_count(v1), HLL_UNION_AGG(v2) from ${tableName2} group by k1, k2 order by k1" + } finally { + try_sq2 "DROP TABLE IF EXISTS ${tableName2}" + } +} + diff --git a/regression-test/suites/load_p0/http_stream/test_http_stream_properties.groovy b/regression-test/suites/load_p0/http_stream/test_http_stream_properties.groovy new file mode 100644 index 00000000000000..b217dcb57d4e29 --- /dev/null +++ b/regression-test/suites/load_p0/http_stream/test_http_stream_properties.groovy @@ -0,0 +1,387 @@ +// 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_http_stream_properties", "p0") { + + def tables = [ + "dup_tbl_basic", + "uniq_tbl_basic", + "mow_tbl_basic", + "agg_tbl_basic", + "dup_tbl_array", + "uniq_tbl_array", + "mow_tbl_array", + ] + + def columns = [ + "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19", + "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19", + "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19", + "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,to_bitmap(c5) c19,HLL_HASH(c5) c20,TO_QUANTILE_STATE(c5,1.0) c21,to_bitmap(c6) c22,HLL_HASH(c6) c23,TO_QUANTILE_STATE(c6,1.0) c24", + "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18", + "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18", + "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18", + ] + + def target_columns = [ + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19,k20,k21,kd19,kd20,kd21", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", + ] + + def timezoneColumns = [ + "k00=unix_timestamp('2007-11-30 10:30:19'),c1,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c1", + "k00=unix_timestamp('2007-11-30 10:30:19'),c1,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c1", + "k00=unix_timestamp('2007-11-30 10:30:19'),c1,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c1", + "k00=unix_timestamp('2007-11-30 10:30:19'),c1,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c1,k19=to_bitmap(c5),k20=HLL_HASH(c5),k21=TO_QUANTILE_STATE(c5,1.0),kd19=to_bitmap(c6),kd20=HLL_HASH(c6),kd21=TO_QUANTILE_STATE(c6,1.0)", + "k00=unix_timestamp('2007-11-30 10:30:19'),c1,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18", + "k00=unix_timestamp('2007-11-30 10:30:19'),c1,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18", + "k00=unix_timestamp('2007-11-30 10:30:19'),c1,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18", + ] + + def files = [ + "../stream_load/basic_data.csv", + "../stream_load/basic_data.csv", + "../stream_load/basic_data.csv", + "../stream_load/basic_data.csv", + "../stream_load/basic_array_data.csv", + "../stream_load/basic_array_data.csv", + "../stream_load/basic_array_data.csv" + ] + + def timezoneFiles = [ + "basic_data_timezone.csv", + "basic_data_timezone.csv", + "basic_data_timezone.csv", + "basic_data_timezone.csv", + "basic_array_data_timezone.csv", + "basic_array_data_timezone.csv", + "basic_array_data_timezone.csv", + ] + + def errorFiles = [ + "basic_data_with_errors.csv", + "basic_data_with_errors.csv", + "basic_data_with_errors.csv", + "basic_data_with_errors.csv", + "basic_array_data_with_errors.csv", + "basic_array_data_with_errors.csv", + "basic_array_data_with_errors.csv", + ] + + // def compress_type = [ + // "gz", + // "bz2", + // "lz4", + // "deflate", + // "lzo", + // ] + + // def compress_files = [ + // "basic_data.csv.gz", + // "basic_data.csv.bz2", + // "basic_data.csv.lz4", + // "basic_data.csv.deflate", + // "basic_data.csv.lzo", + // "basic_array_data.csv.gz", + // "basic_array_data.csv.bz2", + // "basic_array_data.csv.lz4", + // "basic_array_data.csv.deflate", + // "basic_array_data.csv.lzo", + // ] + def compress_files = [ + "dup_tbl_basic": [ + ["../stream_load/basic_data.csv.gz", "gz"], + ["../stream_load/basic_data.csv.bz2", "bz2"], + ], + "uniq_tbl_basic": [ + ["../stream_load/basic_data.csv.gz", "gz"], + ["../stream_load/basic_data.csv.bz2", "bz2"], + ], + "mow_tbl_basic": [ + ["../stream_load/basic_data.csv.gz", "gz"], + ["../stream_load/basic_data.csv.bz2", "bz2"], + ], + "agg_tbl_basic": [ + ["../stream_load/basic_data.csv.gz", "gz"], + ["../stream_load/basic_data.csv.bz2", "bz2"], + ], + "dup_tbl_array": [ + ["../stream_load/basic_array_data.csv.gz", "gz"], + ["../stream_load/basic_array_data.csv.bz2", "bz2"], + ], + "uniq_tbl_array": [ + ["../stream_load/basic_array_data.csv.gz", "gz"], + ["../stream_load/basic_array_data.csv.bz2", "bz2"], + ], + "mow_tbl_array": [ + ["../stream_load/basic_array_data.csv.gz", "gz"], + ["../stream_load/basic_array_data.csv.bz2", "bz2"], + ], + ] + + def loadedRows = [12,12,12,12,8,8,15] + + def filteredRows = [8,8,8,8,12,12,5] + + def maxFilterRatio = [0.4,0.4,0.4,0.4,0.6,0.6,0.6] + + InetSocketAddress address = context.config.feHttpInetSocketAddress + String user = context.config.feHttpUser + String password = context.config.feHttpPassword + String db = context.config.getDbNameByFile(context.file) + + def i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_create.sql""").text + + def tableNm = "stream_load_" + tableName + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableNm}(${target_columns[i]}) select ${columns[i]} from http_stream("format"="csv", "column_separator"="|") + """ + file files[i] + time 10000 // limit inflight 10s + 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(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + i++ + } + } finally { + for (String tableName in tables) { + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_drop.sql""").text + } + } + + // TODO timezone + + // TODO strict_mode + + // TODO max_filter_ratio + + // sequence + try { + sql new File("""${context.file.parent}/../stream_load/ddl/uniq_tbl_basic_drop_sequence.sql""").text + sql new File("""${context.file.parent}/../stream_load/ddl//uniq_tbl_basic_create_sequence.sql""").text + + String tableNm = "stream_load_uniq_tbl_basic_sequence" + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableNm}(${target_columns[0]}) select ${columns[0]} from http_stream("format"="CSV", "column_separator"="|") + """ + file files[0] + time 10000 // limit inflight 10s + + 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(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + qt_sql_squence "select * from stream_load_uniq_tbl_basic_sequence order by k00,k01" + } finally { + sql new File("""${context.file.parent}/../stream_load/ddl/uniq_tbl_basic_drop_sequence.sql""").text + } + + // TODO merge type + + // TODO two_phase_commit + + // compress_type + // gz/bz2 + // TODO lzo/deflate/lz4 + i = 0 + try { + for (String tableName in tables) { + compress_files[tableName].each { fileName, type -> { + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_create.sql""").text + def tableNm = "stream_load_" + tableName + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableNm}(${target_columns[i]}) select ${columns[i]} from http_stream("format"="CSV", "column_separator"="|", "compress_type"="${type}") + """ + file fileName + time 10000 // limit inflight 10s + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${tableName}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + } + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/../stream_load/ddl/${table}_drop.sql""").text + } + } + + // skip_lines + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_create.sql""").text + + def tableNm = "stream_load_" + tableName + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableNm}(${target_columns[i]}) select ${columns[i]} from http_stream("format"="CSV", "column_separator"="|", "skip_lines"="2") + """ + file files[i] + time 10000 // limit inflight 10s + + 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(18, json.NumberTotalRows) + assertEquals(18, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + if (i <= 3) { + qt_sql_skip_lines "select * from ${tableNm} order by k00,k01" + } else { + qt_sql_skip_lines "select * from ${tableNm} order by k00" + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/../stream_load/ddl/${table}_drop.sql""").text + } + } + + // column_separator + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_create.sql""").text + def tableNm = "stream_load_" + tableName + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableNm}(${target_columns[i]}) select ${columns[i]} from http_stream("format"="CSV", "column_separator"=",") + """ + file files[i] + time 10000 // limit inflight 10s + + 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()) + // assertEquals(0, json.NumberTotalRows) + // assertEquals(0, json.NumberLoadedRows) + // assertEquals(0, json.NumberFilteredRows) + // assertEquals(0, json.NumberUnselectedRows) + } + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/../stream_load/ddl/${table}_drop.sql""").text + } + } + + // line_delimiter + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/../stream_load/ddl/${tableName}_create.sql""").text + def tableNm = "stream_load_" + tableName + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableNm}(${target_columns[i]}) select ${columns[i]} from http_stream("format"="CSV", "column_separator"=",", "line_delimiter"=",") + """ + file files[i] + time 10000 // limit inflight 10s + + 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()) + // assertEquals(1, json.NumberTotalRows) + // assertEquals(0, json.NumberLoadedRows) + // assertEquals(1, json.NumberFilteredRows) + // assertEquals(0, json.NumberUnselectedRows) + } + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/../stream_load/ddl/${table}_drop.sql""").text + } + } +} + diff --git a/regression-test/suites/load_p0/stream_load/ddl/agg_tbl_basic_create.sql b/regression-test/suites/load_p0/stream_load/ddl/agg_tbl_basic_create.sql new file mode 100644 index 00000000000000..bb5e1a534ef145 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/agg_tbl_basic_create.sql @@ -0,0 +1,59 @@ +CREATE TABLE stream_load_agg_tbl_basic +( + k00 INT NOT NULL, + k01 DATE NOT NULL, + k02 BOOLEAN REPLACE NULL, + k03 TINYINT SUM NULL, + k04 SMALLINT SUM NULL, + k05 INT SUM NULL, + k06 BIGINT SUM NULL, + k07 LARGEINT SUM NULL, + k08 FLOAT SUM NULL, + k09 DOUBLE SUM NULL, + k10 DECIMAL(9,1) SUM NULL, + k11 DECIMALV3(9,1) SUM NULL, + k12 DATETIME REPLACE NULL, + k13 DATEV2 REPLACE NULL, + k14 DATETIMEV2 REPLACE NULL, + k15 CHAR(300) REPLACE NULL, + k16 VARCHAR(300) REPLACE NULL, + k17 STRING REPLACE NULL, + k18 JSON REPLACE NULL, + k19 BITMAP BITMAP_UNION NULL, + k20 HLL HLL_UNION NULL, + k21 QUANTILE_STATE QUANTILE_UNION NULL, + kd01 BOOLEAN REPLACE NOT NULL DEFAULT "TRUE", + kd02 TINYINT SUM NOT NULL DEFAULT "1", + kd03 SMALLINT SUM NOT NULL DEFAULT "2", + kd04 INT SUM NOT NULL DEFAULT "3", + kd05 BIGINT SUM NOT NULL DEFAULT "4", + kd06 LARGEINT SUM NOT NULL DEFAULT "5", + kd07 FLOAT SUM NOT NULL DEFAULT "6.0", + kd08 DOUBLE SUM NOT NULL DEFAULT "7.0", + kd09 DECIMAL SUM NOT NULL DEFAULT "888888888", + kd10 DECIMALV3 SUM NOT NULL DEFAULT "999999999", + kd11 DATE REPLACE NOT NULL DEFAULT "2023-08-24", + kd12 DATETIME REPLACE NOT NULL DEFAULT "2023-08-24 12:00:00", + kd13 DATEV2 REPLACE NOT NULL DEFAULT "2023-08-24", + kd14 DATETIMEV2 REPLACE NOT NULL DEFAULT "2023-08-24 12:00:00", + kd15 CHAR(300) REPLACE NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd16 VARCHAR(300) REPLACE NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd17 STRING REPLACE NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd18 JSON REPLACE NULL, + kd19 BITMAP BITMAP_UNION NULL, + kd20 HLL HLL_UNION NULL, + kd21 QUANTILE_STATE QUANTILE_UNION NULL, + + INDEX idx_bitmap_k104 (`k01`) USING BITMAP +) +AGGREGATE KEY(k00,k01) +PARTITION BY RANGE(k01) +( + PARTITION p1 VALUES [('2023-08-01'), ('2023-08-11')), + PARTITION p2 VALUES [('2023-08-11'), ('2023-08-21')), + PARTITION p3 VALUES [('2023-08-21'), ('2023-09-01')) +) +DISTRIBUTED BY HASH(k00) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/agg_tbl_basic_drop.sql b/regression-test/suites/load_p0/stream_load/ddl/agg_tbl_basic_drop.sql new file mode 100644 index 00000000000000..ff091742e115fe --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/agg_tbl_basic_drop.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS stream_load_agg_tbl_basic; \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_create.sql b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_create.sql new file mode 100644 index 00000000000000..0277e3368cd216 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_create.sql @@ -0,0 +1,42 @@ +CREATE TABLE stream_load_dup_tbl_array +( + k00 INT NOT NULL, + k01 array NULL, + k02 array NULL, + k03 array NULL, + k04 array NULL, + k05 array NULL, + k06 array NULL, + k07 array NULL, + k08 array NULL, + k09 array NULL, + k10 array NULL, + k11 array NULL, + k12 array NULL, + k13 array NULL, + k14 array NULL, + k15 array NULL, + k16 array NULL, + k17 array NULL, + kd01 array NOT NULL DEFAULT "[]", + kd02 array NOT NULL DEFAULT "[]", + kd03 array NOT NULL DEFAULT "[]", + kd04 array NOT NULL DEFAULT "[]", + kd05 array NOT NULL DEFAULT "[]", + kd06 array NOT NULL DEFAULT "[]", + kd07 array NOT NULL DEFAULT "[]", + kd08 array NOT NULL DEFAULT "[]", + kd09 array NOT NULL DEFAULT "[]", + kd10 array NOT NULL DEFAULT "[]", + kd11 array NOT NULL DEFAULT "[]", + kd12 array NOT NULL DEFAULT "[]", + kd13 array NOT NULL DEFAULT "[]", + kd14 array NOT NULL DEFAULT "[]", + kd15 array NOT NULL DEFAULT "[]", + kd16 array NOT NULL DEFAULT "[]", +) +DUPLICATE KEY(k00) +DISTRIBUTED BY HASH(k00) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_drop.sql b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_drop.sql new file mode 100644 index 00000000000000..b7a0151e091f06 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_array_drop.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS stream_load_dup_tbl_array; \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create.sql b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create.sql new file mode 100644 index 00000000000000..57c5c468c838a1 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create.sql @@ -0,0 +1,65 @@ +CREATE TABLE stream_load_dup_tbl_basic +( + k00 INT NOT NULL, + k01 DATE NOT NULL, + k02 BOOLEAN NULL, + k03 TINYINT NULL, + k04 SMALLINT NULL, + k05 INT NULL, + k06 BIGINT NULL, + k07 LARGEINT NULL, + k08 FLOAT NULL, + k09 DOUBLE NULL, + k10 DECIMAL(9,1) NULL, + k11 DECIMALV3(9,1) NULL, + k12 DATETIME NULL, + k13 DATEV2 NULL, + k14 DATETIMEV2 NULL, + k15 CHAR NULL, + k16 VARCHAR NULL, + k17 STRING NULL, + k18 JSON NULL, + kd01 BOOLEAN NOT NULL DEFAULT "TRUE", + kd02 TINYINT NOT NULL DEFAULT "1", + kd03 SMALLINT NOT NULL DEFAULT "2", + kd04 INT NOT NULL DEFAULT "3", + kd05 BIGINT NOT NULL DEFAULT "4", + kd06 LARGEINT NOT NULL DEFAULT "5", + kd07 FLOAT NOT NULL DEFAULT "6.0", + kd08 DOUBLE NOT NULL DEFAULT "7.0", + kd09 DECIMAL NOT NULL DEFAULT "888888888", + kd10 DECIMALV3 NOT NULL DEFAULT "999999999", + kd11 DATE NOT NULL DEFAULT "2023-08-24", + kd12 DATETIME NOT NULL DEFAULT "2023-08-24 12:00:00", + kd13 DATEV2 NOT NULL DEFAULT "2023-08-24", + kd14 DATETIMEV2 NOT NULL DEFAULT "2023-08-24 12:00:00", + kd15 CHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd16 VARCHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd17 STRING NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd18 JSON NULL, + + INDEX idx_inverted_k104 (`k05`) USING INVERTED, + INDEX idx_inverted_k110 (`k11`) USING INVERTED, + INDEX idx_inverted_k113 (`k13`) USING INVERTED, + INDEX idx_inverted_k114 (`k14`) USING INVERTED, + INDEX idx_inverted_k117 (`k17`) USING INVERTED PROPERTIES("parser" = "english"), + INDEX idx_ngrambf_k115 (`k15`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), + INDEX idx_ngrambf_k116 (`k16`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), + INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), + + INDEX idx_bitmap_k104 (`k02`) USING BITMAP, + INDEX idx_bitmap_k110 (`kd01`) USING BITMAP, + +) +DUPLICATE KEY(k00) +PARTITION BY RANGE(k01) +( + PARTITION p1 VALUES [('2023-08-01'), ('2023-08-11')), + PARTITION p2 VALUES [('2023-08-11'), ('2023-08-21')), + PARTITION p3 VALUES [('2023-08-21'), ('2023-09-01')) +) +DISTRIBUTED BY HASH(k00) BUCKETS 32 +PROPERTIES ( + "bloom_filter_columns"="k05", + "replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create_random_bucket.sql b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create_random_bucket.sql new file mode 100644 index 00000000000000..cbd0fa7e71929a --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_create_random_bucket.sql @@ -0,0 +1,65 @@ +CREATE TABLE stream_load_dup_tbl_basic_random_bucket +( + k00 INT NOT NULL, + k01 DATE NOT NULL, + k02 BOOLEAN NULL, + k03 TINYINT NULL, + k04 SMALLINT NULL, + k05 INT NULL, + k06 BIGINT NULL, + k07 LARGEINT NULL, + k08 FLOAT NULL, + k09 DOUBLE NULL, + k10 DECIMAL(9,1) NULL, + k11 DECIMALV3(9,1) NULL, + k12 DATETIME NULL, + k13 DATEV2 NULL, + k14 DATETIMEV2 NULL, + k15 CHAR NULL, + k16 VARCHAR NULL, + k17 STRING NULL, + k18 JSON NULL, + kd01 BOOLEAN NOT NULL DEFAULT "TRUE", + kd02 TINYINT NOT NULL DEFAULT "1", + kd03 SMALLINT NOT NULL DEFAULT "2", + kd04 INT NOT NULL DEFAULT "3", + kd05 BIGINT NOT NULL DEFAULT "4", + kd06 LARGEINT NOT NULL DEFAULT "5", + kd07 FLOAT NOT NULL DEFAULT "6.0", + kd08 DOUBLE NOT NULL DEFAULT "7.0", + kd09 DECIMAL NOT NULL DEFAULT "888888888", + kd10 DECIMALV3 NOT NULL DEFAULT "999999999", + kd11 DATE NOT NULL DEFAULT "2023-08-24", + kd12 DATETIME NOT NULL DEFAULT "2023-08-24 12:00:00", + kd13 DATEV2 NOT NULL DEFAULT "2023-08-24", + kd14 DATETIMEV2 NOT NULL DEFAULT "2023-08-24 12:00:00", + kd15 CHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd16 VARCHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd17 STRING NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd18 JSON NULL, + + INDEX idx_inverted_k104 (`k05`) USING INVERTED, + INDEX idx_inverted_k110 (`k11`) USING INVERTED, + INDEX idx_inverted_k113 (`k13`) USING INVERTED, + INDEX idx_inverted_k114 (`k14`) USING INVERTED, + INDEX idx_inverted_k117 (`k17`) USING INVERTED PROPERTIES("parser" = "english"), + INDEX idx_ngrambf_k115 (`k15`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), + INDEX idx_ngrambf_k116 (`k16`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), + INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), + + INDEX idx_bitmap_k104 (`k02`) USING BITMAP, + INDEX idx_bitmap_k110 (`kd01`) USING BITMAP, + +) +DUPLICATE KEY(k00) +PARTITION BY RANGE(k01) +( + PARTITION p1 VALUES [('2023-08-01'), ('2023-08-11')), + PARTITION p2 VALUES [('2023-08-11'), ('2023-08-21')), + PARTITION p3 VALUES [('2023-08-21'), ('2023-09-01')) +) +DISTRIBUTED BY random BUCKETS 32 +PROPERTIES ( + "bloom_filter_columns"="k05", + "replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_drop.sql b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_drop.sql new file mode 100644 index 00000000000000..260b70e18a7145 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_drop.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS stream_load_dup_tbl_basic; \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_drop_random_bucket.sql b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_drop_random_bucket.sql new file mode 100644 index 00000000000000..7467a855d0b089 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/dup_tbl_basic_drop_random_bucket.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS stream_load_dup_tbl_basic_random_bucket; \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_array_create.sql b/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_array_create.sql new file mode 100644 index 00000000000000..39bfcfd402886d --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_array_create.sql @@ -0,0 +1,44 @@ +CREATE TABLE stream_load_mow_tbl_array +( + k00 INT NOT NULL, + k01 array NULL, + k02 array NULL, + k03 array NULL, + k04 array NULL, + k05 array NULL, + k06 array NULL, + k07 array NULL, + k08 array NULL, + k09 array NULL, + k10 array NULL, + k11 array NULL, + k12 array NULL, + k13 array NULL, + k14 array NULL, + k15 array NULL, + k16 array NULL, + k17 array NULL, + kd01 array NOT NULL DEFAULT "[]", + kd02 array NOT NULL DEFAULT "[]", + kd03 array NOT NULL DEFAULT "[]", + kd04 array NOT NULL DEFAULT "[]", + kd05 array NOT NULL DEFAULT "[]", + kd06 array NOT NULL DEFAULT "[]", + kd07 array NOT NULL DEFAULT "[]", + kd08 array NOT NULL DEFAULT "[]", + kd09 array NOT NULL DEFAULT "[]", + kd10 array NOT NULL DEFAULT "[]", + kd11 array NOT NULL DEFAULT "[]", + kd12 array NOT NULL DEFAULT "[]", + kd13 array NOT NULL DEFAULT "[]", + kd14 array NOT NULL DEFAULT "[]", + kd15 array NOT NULL DEFAULT "[]", + kd16 array NOT NULL DEFAULT "[]", + kd17 array NOT NULL DEFAULT "[]" +) +UNIQUE KEY(k00) +DISTRIBUTED BY HASH(k00) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1", + "enable_unique_key_merge_on_write" = "true" +); \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_array_drop.sql b/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_array_drop.sql new file mode 100644 index 00000000000000..b98d62a7e88ac2 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_array_drop.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS stream_load_mow_tbl_array; \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_basic_create.sql b/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_basic_create.sql new file mode 100644 index 00000000000000..94695908f875e9 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_basic_create.sql @@ -0,0 +1,67 @@ +CREATE TABLE stream_load_mow_tbl_basic +( + k00 INT NOT NULL, + k01 DATE NULL, + k02 BOOLEAN NULL, + k03 TINYINT NULL, + k04 SMALLINT NULL, + k05 INT NULL, + k06 BIGINT NULL, + k07 LARGEINT NULL, + k08 FLOAT NULL, + k09 DOUBLE NULL, + k10 DECIMAL(9,1) NULL, + k11 DECIMALV3(9,1) NULL, + k12 DATETIME NULL, + k13 DATEV2 NULL, + k14 DATETIMEV2 NULL, + k15 CHAR NULL, + k16 VARCHAR NULL, + k17 STRING NULL, + k18 JSON NULL, + kd01 BOOLEAN NOT NULL DEFAULT "TRUE", + kd02 TINYINT NOT NULL DEFAULT "1", + kd03 SMALLINT NOT NULL DEFAULT "2", + kd04 INT NOT NULL DEFAULT "3", + kd05 BIGINT NOT NULL DEFAULT "4", + kd06 LARGEINT NOT NULL DEFAULT "5", + kd07 FLOAT NOT NULL DEFAULT "6.0", + kd08 DOUBLE NOT NULL DEFAULT "7.0", + kd09 DECIMAL NOT NULL DEFAULT "888888888", + kd10 DECIMALV3 NOT NULL DEFAULT "999999999", + kd11 DATE NOT NULL DEFAULT "2023-08-24", + kd12 DATETIME NOT NULL DEFAULT "2023-08-24 12:00:00", + kd13 DATEV2 NOT NULL DEFAULT "2023-08-24", + kd14 DATETIMEV2 NOT NULL DEFAULT "2023-08-24 12:00:00", + kd15 CHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd16 VARCHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd17 STRING NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd18 JSON NULL, + + INDEX idx_inverted_k104 (`k05`) USING INVERTED, + INDEX idx_inverted_k110 (`k11`) USING INVERTED, + INDEX idx_inverted_k113 (`k13`) USING INVERTED, + INDEX idx_inverted_k114 (`k14`) USING INVERTED, + INDEX idx_inverted_k117 (`k17`) USING INVERTED PROPERTIES("parser" = "english"), + INDEX idx_bitmap_k104 (`k05`) USING BITMAP, + INDEX idx_bitmap_k110 (`k11`) USING BITMAP, + INDEX idx_bitmap_k113 (`k13`) USING BITMAP, + INDEX idx_bitmap_k114 (`k14`) USING BITMAP, + INDEX idx_bitmap_k117 (`k17`) USING BITMAP, + INDEX idx_ngrambf_k115 (`k15`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), + INDEX idx_ngrambf_k116 (`k16`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"), + INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256") +) +UNIQUE KEY(k00,k01) +PARTITION BY RANGE(k01) +( + PARTITION p1 VALUES [('2023-08-01'), ('2023-08-11')), + PARTITION p2 VALUES [('2023-08-11'), ('2023-08-21')), + PARTITION p3 VALUES [('2023-08-21'), ('2023-09-01')) +) +DISTRIBUTED BY HASH(k00) BUCKETS 32 +PROPERTIES ( + "bloom_filter_columns"="k05", + "replication_num" = "1", + "enable_unique_key_merge_on_write" = "true" +); \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_basic_drop.sql b/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_basic_drop.sql new file mode 100644 index 00000000000000..60f0e194130bb3 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/mow_tbl_basic_drop.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS stream_load_mow_tbl_basic; \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_create.sql b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_create.sql new file mode 100644 index 00000000000000..c6ba1b69496409 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_create.sql @@ -0,0 +1,43 @@ +CREATE TABLE stream_load_uniq_tbl_array +( + k00 INT NOT NULL, + k01 array NULL, + k02 array NULL, + k03 array NULL, + k04 array NULL, + k05 array NULL, + k06 array NULL, + k07 array NULL, + k08 array NULL, + k09 array NULL, + k10 array NULL, + k11 array NULL, + k12 array NULL, + k13 array NULL, + k14 array NULL, + k15 array NULL, + k16 array NULL, + k17 array NULL, + kd01 array NOT NULL DEFAULT "[]", + kd02 array NOT NULL DEFAULT "[]", + kd03 array NOT NULL DEFAULT "[]", + kd04 array NOT NULL DEFAULT "[]", + kd05 array NOT NULL DEFAULT "[]", + kd06 array NOT NULL DEFAULT "[]", + kd07 array NOT NULL DEFAULT "[]", + kd08 array NOT NULL DEFAULT "[]", + kd09 array NOT NULL DEFAULT "[]", + kd10 array NOT NULL DEFAULT "[]", + kd11 array NOT NULL DEFAULT "[]", + kd12 array NOT NULL DEFAULT "[]", + kd13 array NOT NULL DEFAULT "[]", + kd14 array NOT NULL DEFAULT "[]", + kd15 array NOT NULL DEFAULT "[]", + kd16 array NOT NULL DEFAULT "[]", + kd17 array NOT NULL DEFAULT "[]", +) +UNIQUE KEY(k00) +DISTRIBUTED BY HASH(k00) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_drop.sql b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_drop.sql new file mode 100644 index 00000000000000..41e238a82aa250 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_array_drop.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS stream_load_uniq_tbl_array; \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_create.sql b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_create.sql new file mode 100644 index 00000000000000..5acd984cc71314 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_create.sql @@ -0,0 +1,57 @@ +CREATE TABLE stream_load_uniq_tbl_basic +( + k00 INT NOT NULL, + k01 DATE NOT NULL, + k02 BOOLEAN NULL, + k03 TINYINT NULL, + k04 SMALLINT NULL, + k05 INT NULL, + k06 BIGINT NULL, + k07 LARGEINT NULL, + k08 FLOAT NULL, + k09 DOUBLE NULL, + k10 DECIMAL(9,1) NULL, + k11 DECIMALV3(9,1) NULL, + k12 DATETIME NULL, + k13 DATEV2 NULL, + k14 DATETIMEV2 NULL, + k15 CHAR NULL, + k16 VARCHAR NULL, + k17 STRING NULL, + k18 JSON NULL, + kd01 BOOLEAN NOT NULL DEFAULT "TRUE", + kd02 TINYINT NOT NULL DEFAULT "1", + kd03 SMALLINT NOT NULL DEFAULT "2", + kd04 INT NOT NULL DEFAULT "3", + kd05 BIGINT NOT NULL DEFAULT "4", + kd06 LARGEINT NOT NULL DEFAULT "5", + kd07 FLOAT NOT NULL DEFAULT "6.0", + kd08 DOUBLE NOT NULL DEFAULT "7.0", + kd09 DECIMAL NOT NULL DEFAULT "888888888", + kd10 DECIMALV3 NOT NULL DEFAULT "999999999", + kd11 DATE NOT NULL DEFAULT "2023-08-24", + kd12 DATETIME NOT NULL DEFAULT "2023-08-24 12:00:00", + kd13 DATEV2 NOT NULL DEFAULT "2023-08-24", + kd14 DATETIMEV2 NOT NULL DEFAULT "2023-08-24 12:00:00", + kd15 CHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd16 VARCHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd17 STRING NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd18 JSON NULL, + + INDEX idx_bitmap_k104 (`k02`) USING BITMAP, + INDEX idx_bitmap_k110 (`kd01`) USING BITMAP, + INDEX idx_bitmap_k113 (`k13`) USING BITMAP, + INDEX idx_bitmap_k114 (`k14`) USING BITMAP, + INDEX idx_bitmap_k117 (`k17`) USING BITMAP +) +UNIQUE KEY(k00,k01) +PARTITION BY RANGE(k01) +( + PARTITION p1 VALUES [('2023-08-01'), ('2023-08-11')), + PARTITION p2 VALUES [('2023-08-11'), ('2023-08-21')), + PARTITION p3 VALUES [('2023-08-21'), ('2023-09-01')) +) +DISTRIBUTED BY HASH(k00) BUCKETS 32 +PROPERTIES ( + "replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_create_sequence.sql b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_create_sequence.sql new file mode 100644 index 00000000000000..7598d8a636b1bb --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_create_sequence.sql @@ -0,0 +1,52 @@ +CREATE TABLE stream_load_uniq_tbl_basic_sequence +( + k00 INT NOT NULL, + k01 DATE NOT NULL, + k02 BOOLEAN NULL, + k03 TINYINT NULL, + k04 SMALLINT NULL, + k05 INT NULL, + k06 BIGINT NULL, + k07 LARGEINT NULL, + k08 FLOAT NULL, + k09 DOUBLE NULL, + k10 DECIMAL(9,1) NULL, + k11 DECIMALV3(9,1) NULL, + k12 DATETIME NULL, + k13 DATEV2 NULL, + k14 DATETIMEV2 NULL, + k15 CHAR NULL, + k16 VARCHAR NULL, + k17 STRING NULL, + k18 JSON NULL, + kd01 BOOLEAN NOT NULL DEFAULT "TRUE", + kd02 TINYINT NOT NULL DEFAULT "1", + kd03 SMALLINT NOT NULL DEFAULT "2", + kd04 INT NOT NULL DEFAULT "3", + kd05 BIGINT NOT NULL DEFAULT "4", + kd06 LARGEINT NOT NULL DEFAULT "5", + kd07 FLOAT NOT NULL DEFAULT "6.0", + kd08 DOUBLE NOT NULL DEFAULT "7.0", + kd09 DECIMAL NOT NULL DEFAULT "888888888", + kd10 DECIMALV3 NOT NULL DEFAULT "999999999", + kd11 DATE NOT NULL DEFAULT "2023-08-24", + kd12 DATETIME NOT NULL DEFAULT "2023-08-24 12:00:00", + kd13 DATEV2 NOT NULL DEFAULT "2023-08-24", + kd14 DATETIMEV2 NOT NULL DEFAULT "2023-08-24 12:00:00", + kd15 CHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd16 VARCHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd17 STRING NOT NULL DEFAULT "我能吞下玻璃而不伤身体", + kd18 JSON NULL, + + INDEX idx_bitmap_k104 (`k02`) USING BITMAP, + INDEX idx_bitmap_k110 (`kd01`) USING BITMAP, + INDEX idx_bitmap_k113 (`k13`) USING BITMAP, + INDEX idx_bitmap_k114 (`k14`) USING BITMAP, + INDEX idx_bitmap_k117 (`k17`) USING BITMAP +) +UNIQUE KEY(k00,k01) +DISTRIBUTED BY HASH(k00) BUCKETS 32 +PROPERTIES ( + "function_column.sequence_col" = 'k00', + "replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_drop.sql b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_drop.sql new file mode 100644 index 00000000000000..1ee47bb4143cf6 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_drop.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS stream_load_uniq_tbl_basic; diff --git a/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_drop_sequence.sql b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_drop_sequence.sql new file mode 100644 index 00000000000000..92c5c5be56d110 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/ddl/uniq_tbl_basic_drop_sequence.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS stream_load_uniq_tbl_basic_sequence; diff --git a/regression-test/suites/load_p0/stream_load/test_group_commit_stream_load.groovy b/regression-test/suites/load_p0/stream_load/test_group_commit_stream_load.groovy new file mode 100644 index 00000000000000..9e50aebf644474 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/test_group_commit_stream_load.groovy @@ -0,0 +1,310 @@ +// 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_group_commit_stream_load") { + def tableName = "test_group_commit_stream_load" + + def getRowCount = { expectedRowCount -> + def retry = 0 + while (retry < 10) { + sleep(2000) + def rowCount = sql "select count(*) from ${tableName}" + logger.info("rowCount: " + rowCount + ", retry: " + retry) + if (rowCount[0][0] >= expectedRowCount) { + break + } + retry++ + } + } + + def getAlterTableState = { + def retry = 0 + while (true) { + sleep(8000) + def state = sql "show alter table column where tablename = '${tableName}' order by CreateTime desc " + logger.info("alter table retry: ${retry}, state: ${state}") + if (state.size() > 0 && state[0][9] == "FINISHED") { + return true + } + retry++ + if (retry >= 40) { + return false + } + } + return false + } + + try { + // create table + sql """ drop table if exists ${tableName}; """ + + sql """ + CREATE TABLE `${tableName}` ( + `id` int(11) NOT NULL, + `name` varchar(1100) NULL, + `score` int(11) NULL default "-1" + ) ENGINE=OLAP + DUPLICATE KEY(`id`, `name`) + PARTITION BY RANGE (id) ( + PARTITION plessThan1 VALUES LESS THAN ("0"), + PARTITION plessThan2 VALUES LESS THAN ("100") + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + // stream load with compress file + String[] compressionTypes = new String[]{"gz", "bz2", /*"lzo",*/ "lz4"} //, "deflate"} + for (final def compressionType in compressionTypes) { + def fileName = "test_compress.csv." + compressionType + streamLoad { + table "${tableName}" + + set 'column_separator', ',' + set 'group_commit', 'true' + set 'compress_type', "${compressionType}" + // set 'columns', 'id, name, score' + file "${fileName}" + unset 'label' + + time 10000 // limit inflight 10s + } + } + + // stream load with 2 columns + streamLoad { + table "${tableName}" + + set 'column_separator', ',' + set 'group_commit', 'true' + set 'columns', 'id, name' + file "test_stream_load1.csv" + unset 'label' + + time 10000 // limit inflight 10s + } + + // stream load with different column order + streamLoad { + table "${tableName}" + + set 'column_separator', '|' + set 'group_commit', 'true' + set 'columns', 'score, id, name' + file "test_stream_load2.csv" + unset 'label' + + time 10000 // limit inflight 10s + } + + // stream load with where condition + streamLoad { + table "${tableName}" + + set 'column_separator', ',' + set 'group_commit', 'true' + set 'columns', 'id, name' + file "test_stream_load1.csv" + set 'where', 'id > 5' + unset 'label' + + time 10000 // limit inflight 10s + + 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()) + assertTrue(json.GroupCommit) + assertEquals(2, json.NumberTotalRows) + assertEquals(1, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(1, json.NumberUnselectedRows) + } + } + + // stream load with mapping + streamLoad { + table "${tableName}" + + set 'column_separator', ',' + set 'group_commit', 'true' + set 'columns', 'id, name, score = id * 10' + file "test_stream_load1.csv" + unset 'label' + + time 10000 // limit inflight 10s + } + + // stream load with filtered rows + streamLoad { + table "${tableName}" + + set 'column_separator', ',' + set 'group_commit', 'true' + file "test_stream_load3.csv" + set 'where', "name = 'a'" + set 'max_filter_ratio', '0.7' + unset 'label' + + time 10000 // limit inflight 10s + + 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()) + assertTrue(json.GroupCommit) + assertEquals(6, json.NumberTotalRows) + assertEquals(2, json.NumberLoadedRows) + assertEquals(3, json.NumberFilteredRows) + assertEquals(1, json.NumberUnselectedRows) + assertFalse(json.ErrorURL.isEmpty()) + } + } + + // stream load with label + streamLoad { + table "${tableName}" + + // set 'label', 'test_stream_load' + set 'column_separator', '|' + set 'group_commit', 'true' + // set 'label', 'l_' + System.currentTimeMillis() + file "test_stream_load2.csv" + + time 10000 // limit inflight 10s + 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()) + } + } + + getRowCount(21) + qt_sql " SELECT * FROM ${tableName} order by id, name, score asc; " + } finally { + // try_sql("DROP TABLE ${tableName}") + } + + // stream load with large data and schema change + tableName = "test_stream_load_lineorder" + try { + sql """ DROP TABLE IF EXISTS `${tableName}` """ + sql """ + CREATE TABLE IF NOT EXISTS `${tableName}` ( + `lo_orderkey` bigint(20) NOT NULL COMMENT "", + `lo_linenumber` bigint(20) NOT NULL COMMENT "", + `lo_custkey` int(11) NOT NULL COMMENT "", + `lo_partkey` int(11) NOT NULL COMMENT "", + `lo_suppkey` int(11) NOT NULL COMMENT "", + `lo_orderdate` int(11) NOT NULL COMMENT "", + `lo_orderpriority` varchar(16) NOT NULL COMMENT "", + `lo_shippriority` int(11) NOT NULL COMMENT "", + `lo_quantity` bigint(20) NOT NULL COMMENT "", + `lo_extendedprice` bigint(20) NOT NULL COMMENT "", + `lo_ordtotalprice` bigint(20) NOT NULL COMMENT "", + `lo_discount` bigint(20) NOT NULL COMMENT "", + `lo_revenue` bigint(20) NOT NULL COMMENT "", + `lo_supplycost` bigint(20) NOT NULL COMMENT "", + `lo_tax` bigint(20) NOT NULL COMMENT "", + `lo_commitdate` bigint(20) NOT NULL COMMENT "", + `lo_shipmode` varchar(11) NOT NULL COMMENT "" + ) + PARTITION BY RANGE(`lo_orderdate`) + (PARTITION p1992 VALUES [("-2147483648"), ("19930101")), + PARTITION p1993 VALUES [("19930101"), ("19940101")), + PARTITION p1994 VALUES [("19940101"), ("19950101")), + PARTITION p1995 VALUES [("19950101"), ("19960101")), + PARTITION p1996 VALUES [("19960101"), ("19970101")), + PARTITION p1997 VALUES [("19970101"), ("19980101")), + PARTITION p1998 VALUES [("19980101"), ("19990101"))) + DISTRIBUTED BY HASH(`lo_orderkey`) BUCKETS 4 + PROPERTIES ( + "replication_num" = "1" + ); + """ + // load data + def columns = """lo_orderkey,lo_linenumber,lo_custkey,lo_partkey,lo_suppkey,lo_orderdate,lo_orderpriority, + lo_shippriority,lo_quantity,lo_extendedprice,lo_ordtotalprice,lo_discount, + lo_revenue,lo_supplycost,lo_tax,lo_commitdate,lo_shipmode""" + + new Thread(() -> { + Thread.sleep(3000) + // do light weight schema change + sql """ alter table ${tableName} ADD column sc_tmp varchar(100) after lo_revenue; """ + + assertTrue(getAlterTableState()) + + // do hard weight schema change + def new_columns = """lo_orderkey,lo_linenumber,lo_custkey,lo_partkey,lo_suppkey,lo_orderdate,lo_orderpriority, + lo_shippriority,lo_quantity,lo_extendedprice,lo_ordtotalprice,lo_discount, + lo_revenue,lo_supplycost,lo_tax,lo_shipmode,lo_commitdate""" + sql """ alter table ${tableName} order by (${new_columns}); """ + }).start(); + + for (int i = 0; i < 4; i++) { + + streamLoad { + table tableName + + set 'column_separator', '|' + set 'compress_type', 'GZ' + set 'columns', columns + ",lo_dummy" + set 'group_commit', 'true' + unset 'label' + + file """${getS3Url()}/regression/ssb/sf0.1/lineorder.tbl.gz""" + + time 10000 // limit inflight 10s + + // stream load action will check result, include Success status, and NumberTotalRows == NumberLoadedRows + + // 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 ${i}, result: ${result}") + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(json.NumberTotalRows, json.NumberLoadedRows) + assertEquals(json.NumberLoadedRows, 600572) + assertTrue(json.LoadBytes > 0) + assertTrue(json.GroupCommit) + } + } + } + + getRowCount(2402288) + qt_sql """ select count(*) from ${tableName} """ + + assertTrue(getAlterTableState()) + } finally { + // try_sql("DROP TABLE ${tableName}") + } +} \ No newline at end of file diff --git a/regression-test/suites/load_p0/stream_load/test_stream_load_properties.groovy b/regression-test/suites/load_p0/stream_load/test_stream_load_properties.groovy new file mode 100644 index 00000000000000..2b8d56a7a47a60 --- /dev/null +++ b/regression-test/suites/load_p0/stream_load/test_stream_load_properties.groovy @@ -0,0 +1,739 @@ +// 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.apache.http.HttpRequest; +import org.apache.http.HttpResponse; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.client.methods.RequestBuilder; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.protocol.HttpContext; +import org.apache.http.util.EntityUtils; +import org.apache.http.client.RedirectStrategy; +import org.apache.http.impl.client.LaxRedirectStrategy; + +suite("test_stream_load_properties", "p0") { + + def tables = [ + "dup_tbl_basic", + "uniq_tbl_basic", + "mow_tbl_basic", + "agg_tbl_basic", + "dup_tbl_array", + "uniq_tbl_array", + "mow_tbl_array", + ] + + def columns = [ + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19=to_bitmap(k04),k20=HLL_HASH(k04),k21=TO_QUANTILE_STATE(k04,1.0),kd19=to_bitmap(k05),kd20=HLL_HASH(k05),kd21=TO_QUANTILE_STATE(k05,1.0)", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", + "k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", + ] + + def timezoneColumns = + [ + "k00=unix_timestamp('2007-11-30 10:30:19'),k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", + "k00=unix_timestamp('2007-11-30 10:30:19'),k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", + "k00=unix_timestamp('2007-11-30 10:30:19'),k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18", + "k00=unix_timestamp('2007-11-30 10:30:19'),k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18,k19=to_bitmap(k04),k20=HLL_HASH(k04),k21=TO_QUANTILE_STATE(k04,1.0),kd19=to_bitmap(k05),kd20=HLL_HASH(k05),kd21=TO_QUANTILE_STATE(k05,1.0)", + "k00=unix_timestamp('2007-11-30 10:30:19'),k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", + "k00=unix_timestamp('2007-11-30 10:30:19'),k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", + "k00=unix_timestamp('2007-11-30 10:30:19'),k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17", + ] + + def files = [ + "basic_data.csv", + "basic_data.csv", + "basic_data.csv", + "basic_data.csv", + "basic_array_data.csv", + "basic_array_data.csv", + "basic_array_data.csv" + ] + + def timezoneFiles = [ + "basic_data_timezone.csv", + "basic_data_timezone.csv", + "basic_data_timezone.csv", + "basic_data_timezone.csv", + "basic_array_data_timezone.csv", + "basic_array_data_timezone.csv", + "basic_array_data_timezone.csv", + ] + + def errorFiles = [ + "basic_data_with_errors.csv", + "basic_data_with_errors.csv", + "basic_data_with_errors.csv", + "basic_data_with_errors.csv", + "basic_array_data_with_errors.csv", + "basic_array_data_with_errors.csv", + "basic_array_data_with_errors.csv", + ] + + def compress_type = [ + "gz", + "bz2", + "lz4", + "deflate", + "lzo", + ] + + def compress_files = [ + "basic_data.csv.gz", + "basic_data.csv.bz2", + "basic_data.csv.lz4", + "basic_data.csv.deflate", + "basic_data.csv.lzo", + "basic_array_data.csv.gz", + "basic_array_data.csv.bz2", + "basic_array_data.csv.lz4", + "basic_array_data.csv.deflate", + "basic_array_data.csv.lzo", + ] + + def loadedRows = [12,12,12,12,8,8,15] + + def filteredRows = [8,8,8,8,12,12,5] + + def maxFilterRatio = [0.4,0.4,0.4,0.4,0.6,0.6,0.6] + + InetSocketAddress address = context.config.feHttpInetSocketAddress + String user = context.config.feHttpUser + String password = context.config.feHttpPassword + String db = context.config.getDbNameByFile(context.file) + + // exec_mem_limit + def i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', columns[i] + //set 'exec_mem_limit', '1' + file files[i] + time 10000 // limit inflight 10s + + 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(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + i++ + } + } finally { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + } + } + + // timezone + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', timezoneColumns[i] + set 'timezone', 'Asia/Shanghai' + file timezoneFiles[i] + time 10000 // limit inflight 10s + + 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(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + def tableName1 = "stream_load_" + tableName + if (i <= 3) { + qt_sql_timezone_shanghai "select * from ${tableName1} order by k00,k01" + } else { + qt_sql_timezone_shanghai "select * from ${tableName1} order by k00" + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_drop.sql""").text + } + } + + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', timezoneColumns[i] + set 'timezone', 'Africa/Abidjan' + file timezoneFiles[i] + time 10000 // limit inflight 10s + + 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(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + def tableName1 = "stream_load_" + tableName + if (i <= 3) { + qt_sql_timezone_Abidjan "select * from ${tableName1} order by k00,k01" + } else { + qt_sql_timezone_Abidjan "select * from ${tableName1} order by k00" + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_drop.sql""").text + } + } + + // strict_mode + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', columns[i] + set 'strict_mode', 'true' + file errorFiles[i] + time 10000 // limit inflight 10s + + 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()) + assertEquals(20, json.NumberTotalRows) + assertEquals(loadedRows[i], json.NumberLoadedRows) + assertEquals(filteredRows[i], json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + def tableName1 = "stream_load_" + tableName + if (i <= 3) { + qt_sql_strict_mode "select * from ${tableName1} order by k00,k01" + } else { + qt_sql_strict_mode "select * from ${tableName1} order by k00" + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_drop.sql""").text + } + } + + // max_filter_ratio + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', columns[i] + set 'strict_mode', 'true' + set 'max_filter_ratio', "${maxFilterRatio[i]}" + file errorFiles[i] + time 10000 // limit inflight 10s + + 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(20, json.NumberTotalRows) + assertEquals(loadedRows[i], json.NumberLoadedRows) + assertEquals(filteredRows[i], json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + def tableName1 = "stream_load_" + tableName + if (i <= 3) { + qt_sql_max_filter_ratio "select * from ${tableName1} order by k00,k01" + } else { + qt_sql_max_filter_ratio "select * from ${tableName1} order by k00" + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_drop.sql""").text + } + } + + // load_to_single_tablet + try { + sql new File("""${context.file.parent}/ddl/dup_tbl_basic_drop_random_bucket.sql""").text + sql new File("""${context.file.parent}/ddl/dup_tbl_basic_create_random_bucket.sql""").text + + streamLoad { + table 'stream_load_dup_tbl_basic_random_bucket' + set 'column_separator', '|' + set 'columns', columns[0] + set 'load_to_single_tablet', 'true' + file files[0] + time 10000 // limit inflight 10s + + 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(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + // def res = sql "show tablets from stream_load_dup_tbl_basic_random_bucket" + // assertEquals(res[0][10].toString(), "20") + } finally { + sql new File("""${context.file.parent}/ddl/dup_tbl_basic_drop_random_bucket.sql""").text + } + + // sequence + try { + sql new File("""${context.file.parent}/ddl/uniq_tbl_basic_drop_sequence.sql""").text + sql new File("""${context.file.parent}/ddl/uniq_tbl_basic_create_sequence.sql""").text + + streamLoad { + table 'stream_load_uniq_tbl_basic_sequence' + set 'column_separator', '|' + set 'columns', columns[0] + file files[0] + time 10000 // limit inflight 10s + + 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(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + qt_sql_squence "select * from stream_load_uniq_tbl_basic_sequence order by k00,k01" + } finally { + sql new File("""${context.file.parent}/ddl/uniq_tbl_basic_drop_sequence.sql""").text + } + + // merge type + i = 0 + try { + def tableName = "mow_tbl_basic" + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', columns[i] + file files[i] + time 10000 // limit inflight 10s + + 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(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', columns[i] + set 'merge_type', 'DELETE' + file files[i] + time 10000 // limit inflight 10s + + 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(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + def tableName1 = "stream_load_" + tableName + qt_sql_merge_type "select * from ${tableName1} order by k00,k01" + } finally { + sql new File("""${context.file.parent}/ddl/mow_tbl_basic_drop.sql""").text + } + + // two_phase_commit + def do_streamload_2pc = { txn_id, txn_operation, tableName-> + HttpClients.createDefault().withCloseable { client -> + RequestBuilder requestBuilder = RequestBuilder.put("http://${address.hostString}:${address.port}/api/${db}/${tableName}/_stream_load_2pc") + String encoding = Base64.getEncoder() + .encodeToString((user + ":" + (password == null ? "" : password)).getBytes("UTF-8")) + requestBuilder.setHeader("Authorization", "Basic ${encoding}") + requestBuilder.setHeader("Expect", "100-Continue") + requestBuilder.setHeader("txn_id", "${txn_id}") + requestBuilder.setHeader("txn_operation", "${txn_operation}") + + String backendStreamLoadUri = null + client.execute(requestBuilder.build()).withCloseable { resp -> + resp.withCloseable { + String body = EntityUtils.toString(resp.getEntity()) + def respCode = resp.getStatusLine().getStatusCode() + // should redirect to backend + if (respCode != 307) { + throw new IllegalStateException("Expect frontend stream load response code is 307, " + + "but meet ${respCode}\nbody: ${body}") + } + backendStreamLoadUri = resp.getFirstHeader("location").getValue() + } + } + + requestBuilder.setUri(backendStreamLoadUri) + try{ + client.execute(requestBuilder.build()).withCloseable { resp -> + resp.withCloseable { + String body = EntityUtils.toString(resp.getEntity()) + def respCode = resp.getStatusLine().getStatusCode() + if (respCode != 200) { + throw new IllegalStateException("Expect backend stream load response code is 200, " + + "but meet ${respCode}\nbody: ${body}") + } + } + } + } catch (Throwable t) { + log.info("StreamLoad Exception: ", t) + } + } + } + + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + String txnId + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', columns[i] + set 'two_phase_commit', 'true' + file files[i] + time 10000 // limit inflight 10s + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + txnId = json.TxnId + assertEquals("success", json.Status.toLowerCase()) + assertEquals(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + + def tableName1 = "stream_load_" + tableName + if (i <= 3) { + qt_sql_2pc "select * from ${tableName1} order by k00,k01" + } else { + qt_sql_2pc "select * from ${tableName1} order by k00" + } + + do_streamload_2pc.call(txnId, "abort", tableName1) + + if (i <= 3) { + qt_sql_2pc_abort "select * from ${tableName1} order by k00,k01" + } else { + qt_sql_2pc_abort "select * from ${tableName1} order by k00" + } + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', columns[i] + set 'two_phase_commit', 'true' + file files[i] + time 10000 // limit inflight 10s + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + txnId = json.TxnId + assertEquals("success", json.Status.toLowerCase()) + assertEquals(20, json.NumberTotalRows) + assertEquals(20, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + + do_streamload_2pc.call(txnId, "commit", tableName1) + + sleep(60) + if (i <= 3) { + qt_sql_2pc_commit "select * from ${tableName1} order by k00,k01" + } else { + qt_sql_2pc_commit "select * from ${tableName1} order by k00" + } + + i++ + } + } finally { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + } + } + + // compress_type + // gz/bz2/lz4 + // todo lzo/deflate + // i = 0 + // try { + // for (String tableName in tables) { + // for (int j = 0; j < 3 ; j++) { + // sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + // sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + // streamLoad { + // table "stream_load_" + tableName + // set 'column_separator', '|' + // set 'columns', columns[i] + // set 'compress_type', compress_type[j] + // if (i <= 3) { + // file compress_files[0+j] + // }else{ + // file compress_files[5+j] + // } + // time 10000 // limit inflight 10s + + // check { result, exception, startTime, endTime -> + // if (exception != null) { + // throw exception + // } + // log.info("Stream load result: ${tableName}".toString()) + // def json = parseJson(result) + // assertEquals("success", json.Status.toLowerCase()) + // assertEquals(20, json.NumberTotalRows) + // assertEquals(20, json.NumberLoadedRows) + // assertEquals(0, json.NumberFilteredRows) + // assertEquals(0, json.NumberUnselectedRows) + // } + // } + // def tableName1 = "stream_load_" + tableName + // if (i <= 3) { + // qt_sql_compress_type "select * from ${tableName1} order by k00,k01" + // } else { + // qt_sql_compress_type "select * from ${tableName1} order by k00" + // } + // } + // i++ + // } + // } finally { + // for (String table in tables) { + // sql new File("""${context.file.parent}/ddl/${table}_drop.sql""").text + // } + // } + + // skip_lines + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'columns', columns[i] + set 'skip_lines', '2' + file files[i] + time 10000 // limit inflight 10s + + 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(18, json.NumberTotalRows) + assertEquals(18, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + def tableName1 = "stream_load_" + tableName + if (i <= 3) { + qt_sql_skip_lines "select * from ${tableName1} order by k00,k01" + } else { + qt_sql_skip_lines "select * from ${tableName1} order by k00" + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_drop.sql""").text + } + } + + // column_separator + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', ',' + set 'columns', columns[i] + file files[i] + time 10000 // limit inflight 10s + + 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()) + assertEquals(20, json.NumberTotalRows) + assertEquals(0, json.NumberLoadedRows) + assertEquals(20, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_drop.sql""").text + } + } + + // line_delimiter + i = 0 + try { + for (String tableName in tables) { + sql new File("""${context.file.parent}/ddl/${tableName}_drop.sql""").text + sql new File("""${context.file.parent}/ddl/${tableName}_create.sql""").text + + streamLoad { + table "stream_load_" + tableName + set 'column_separator', '|' + set 'line_delimiter', 'line_delimiter' + set 'columns', columns[i] + file files[i] + time 10000 // limit inflight 10s + + 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()) + assertEquals(1, json.NumberTotalRows) + assertEquals(0, json.NumberLoadedRows) + assertEquals(1, json.NumberFilteredRows) + assertEquals(0, json.NumberUnselectedRows) + } + } + i++ + } + } finally { + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_drop.sql""").text + } + } +} diff --git a/regression-test/suites/nereids_function_p0/scalar_function/J.groovy b/regression-test/suites/nereids_function_p0/scalar_function/J.groovy index 1f743333e70afe..939e70e5db6191 100644 --- a/regression-test/suites/nereids_function_p0/scalar_function/J.groovy +++ b/regression-test/suites/nereids_function_p0/scalar_function/J.groovy @@ -20,6 +20,10 @@ 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" + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + // define a sql table def testTable = "tbl_test_jsonb" def dataFile = "test_jsonb.csv" diff --git a/regression-test/suites/nereids_p0/join/test_left_join_null.groovy b/regression-test/suites/nereids_p0/join/test_left_join_null.groovy index ba15a1f831d4e2..970974022d7740 100644 --- a/regression-test/suites/nereids_p0/join/test_left_join_null.groovy +++ b/regression-test/suites/nereids_p0/join/test_left_join_null.groovy @@ -21,6 +21,9 @@ suite("test_left_join_null", "query") { def tbName1 = "dept_emp" def tbName2 = "departments" + + sql "drop table if exists ${tbName1}" + sql "drop table if exists ${tbName2}" sql """ CREATE TABLE IF NOT EXISTS ${tbName1} ( 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 index 14047a678b8dd1..e16c8a6e96005e 100644 --- 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 @@ -20,6 +20,10 @@ 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" + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + // define a sql table def testTable = "tbl_test_json" def dataFile = "test_json.csv" 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 index 87916fddc29e0a..4aafc86b566359 100644 --- 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 @@ -18,6 +18,10 @@ suite("test_json_unique_load_and_function", "p0") { sql "SET enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + // define a sql table def testTable = "tbl_test_json_unique" def dataFile = "test_json_unique_key.csv" 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 index 33258d6bec5bbc..685acae23248cb 100644 --- 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 @@ -20,6 +20,10 @@ 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" + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + // define a sql table def testTable = "tbl_test_jsonb" def dataFile = "test_jsonb.csv" 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 index 013f5d6fab0581..70b5655a1c4481 100644 --- 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 @@ -18,6 +18,10 @@ suite("test_jsonb_unique_load_and_function", "p0") { sql "SET enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + // define a sql table def testTable = "tbl_test_jsonb_unique" def dataFile = "test_jsonb_unique_key.csv" diff --git a/regression-test/suites/nereids_syntax_p0/test_simplify_comparison.groovy b/regression-test/suites/nereids_syntax_p0/test_simplify_comparison.groovy index 4b3cd3bdca1821..9a0a2c5bf59d6a 100644 --- a/regression-test/suites/nereids_syntax_p0/test_simplify_comparison.groovy +++ b/regression-test/suites/nereids_syntax_p0/test_simplify_comparison.groovy @@ -19,57 +19,6 @@ suite("test_simplify_comparison") { sql "set enable_nereids_planner=true" sql 'set enable_fallback_to_original_planner=false;' sql 'drop table if exists log_items_test' - sql """CREATE TABLE IF NOT EXISTS `log_items_test` ( - a DATETIME NOT NULL, - b decimal(10,2) - ) ENGINE=OLAP - UNIQUE KEY (`a`) - DISTRIBUTED BY HASH(`a`) BUCKETS 120 - PROPERTIES ( - "replication_num" = "1", - "in_memory" = "false", - "compression" = "LZ4", - "storage_cooldown_time" = "9999-12-31 23:59:59", - "enable_unique_key_merge_on_write" = "true" - );""" - sql """insert into log_items_test values( "2023-06-06", 111.11 );""" - - explain { - sql "verbose select * from log_items_test where a < '2023-06-15 23:59:59.999' and b < 111.111;" - notContains "CAST" - contains "< 111.12" - contains "< '2023-06-16 00:00:00'" - } - - explain { - sql "verbose select * from log_items_test where a <= '2023-06-15 23:59:59.999' and b <= 111.111;" - notContains "CAST" - contains "<= 111.11" - contains "<= '2023-06-15 23:59:59'" - } - - explain { - sql "verbose select * from log_items_test where a = '2023-06-15 23:59:59.999' and b = 111.111;" - notContains "CAST" - notContains "111.12" - notContains "2023-06-16 00:00:00" - notContains "111.11" - notContains "2023-06-15 23:59:59" - } - - explain { - sql "verbose select * from log_items_test where a > '2023-06-15 23:59:59.999' and b > 111.111;" - notContains "CAST" - contains "> 111.11" - contains "> '2023-06-15 23:59:59'" - } - - explain { - sql "verbose select * from log_items_test where a >= '2023-06-15 23:59:59.999' and b >= 111.111;" - notContains "CAST" - contains ">= 111.12" - contains ">= '2023-06-16 00:00:00'" - } sql "select cast('1234' as decimalv3(18,4)) > 2000;" diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/load.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/load.groovy deleted file mode 100644 index 7018c400379a5c..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/load.groovy +++ /dev/null @@ -1,467 +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. - -suite("load") { - String database = context.config.getDbNameByFile(context.file) - sql "drop database if exists ${database}" - sql "create database ${database}" - sql "use ${database}" - sql """ - drop table if exists lineitem; - """ - sql """ - CREATE TABLE lineitem ( - l_shipdate DATEV2 NOT NULL, - l_orderkey bigint NOT NULL, - l_linenumber int not null, - l_partkey int NOT NULL, - l_suppkey int not null, - l_quantity decimal(15, 2) NOT NULL, - l_extendedprice decimal(15, 2) NOT NULL, - l_discount decimal(15, 2) NOT NULL, - l_tax decimal(15, 2) NOT NULL, - l_returnflag VARCHAR(1) NOT NULL, - l_linestatus VARCHAR(1) NOT NULL, - l_commitdate DATEV2 NOT NULL, - l_receiptdate DATEV2 NOT NULL, - l_shipinstruct VARCHAR(25) NOT NULL, - l_shipmode VARCHAR(10) NOT NULL, - l_comment VARCHAR(44) NOT NULL - )ENGINE=OLAP - DUPLICATE KEY(`l_shipdate`, `l_orderkey`) - COMMENT "OLAP" - DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 - PROPERTIES ( - "replication_num" = "1", - "colocate_with" = "lineitem_orders" - ); - """ - - sql """ - drop table if exists orders; - """ - - sql ''' - CREATE TABLE orders ( - o_orderkey bigint NOT NULL, - o_orderdate DATEV2 NOT NULL, - o_custkey int NOT NULL, - o_orderstatus VARCHAR(1) NOT NULL, - o_totalprice decimal(15, 2) NOT NULL, - o_orderpriority VARCHAR(15) NOT NULL, - o_clerk VARCHAR(15) NOT NULL, - o_shippriority int NOT NULL, - o_comment VARCHAR(79) NOT NULL - )ENGINE=OLAP - DUPLICATE KEY(`o_orderkey`, `o_orderdate`) - COMMENT "OLAP" - DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 - PROPERTIES ( - "replication_num" = "1", - "colocate_with" = "lineitem_orders" - ); ''' - - sql ''' - drop table if exists partsupp; - ''' - - sql ''' - CREATE TABLE partsupp ( - ps_partkey int NOT NULL, - ps_suppkey int NOT NULL, - ps_availqty int NOT NULL, - ps_supplycost decimal(15, 2) NOT NULL, - ps_comment VARCHAR(199) NOT NULL - )ENGINE=OLAP - DUPLICATE KEY(`ps_partkey`) - COMMENT "OLAP" - DISTRIBUTED BY HASH(`ps_partkey`) BUCKETS 24 - PROPERTIES ( - "replication_num" = "1", - "colocate_with" = "part_partsupp" - ); - ''' - - sql ''' - drop table if exists part; - ''' - - sql ''' - CREATE TABLE part ( - p_partkey int NOT NULL, - p_name VARCHAR(55) NOT NULL, - p_mfgr VARCHAR(25) NOT NULL, - p_brand VARCHAR(10) NOT NULL, - p_type VARCHAR(25) NOT NULL, - p_size int NOT NULL, - p_container VARCHAR(10) NOT NULL, - p_retailprice decimal(15, 2) NOT NULL, - p_comment VARCHAR(23) NOT NULL - )ENGINE=OLAP - DUPLICATE KEY(`p_partkey`) - COMMENT "OLAP" - DISTRIBUTED BY HASH(`p_partkey`) BUCKETS 24 - PROPERTIES ( - "replication_num" = "1", - "colocate_with" = "part_partsupp" - ); - ''' - - sql ''' - drop table if exists customer; - ''' - - sql ''' - CREATE TABLE customer ( - c_custkey int NOT NULL, - c_name VARCHAR(25) NOT NULL, - c_address VARCHAR(40) NOT NULL, - c_nationkey int NOT NULL, - c_phone VARCHAR(15) NOT NULL, - c_acctbal decimal(15, 2) NOT NULL, - c_mktsegment VARCHAR(10) NOT NULL, - c_comment VARCHAR(117) NOT NULL - )ENGINE=OLAP - DUPLICATE KEY(`c_custkey`) - COMMENT "OLAP" - DISTRIBUTED BY HASH(`c_custkey`) BUCKETS 24 - PROPERTIES ( - "replication_num" = "1" - ); - ''' - - sql ''' - drop table if exists supplier - ''' - - sql ''' - CREATE TABLE supplier ( - s_suppkey int NOT NULL, - s_name VARCHAR(25) NOT NULL, - s_address VARCHAR(40) NOT NULL, - s_nationkey int NOT NULL, - s_phone VARCHAR(15) NOT NULL, - s_acctbal decimal(15, 2) NOT NULL, - s_comment VARCHAR(101) NOT NULL - )ENGINE=OLAP - DUPLICATE KEY(`s_suppkey`) - COMMENT "OLAP" - DISTRIBUTED BY HASH(`s_suppkey`) BUCKETS 12 - PROPERTIES ( - "replication_num" = "1" - ); - ''' - - sql ''' - drop table if exists nation; - ''' - - sql ''' - CREATE TABLE `nation` ( - `n_nationkey` int(11) NOT NULL, - `n_name` varchar(25) NOT NULL, - `n_regionkey` int(11) NOT NULL, - `n_comment` varchar(152) NULL - ) ENGINE=OLAP - DUPLICATE KEY(`N_NATIONKEY`) - COMMENT "OLAP" - DISTRIBUTED BY HASH(`N_NATIONKEY`) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ); - ''' - - sql ''' - drop table if exists region; - ''' - - sql ''' - CREATE TABLE region ( - r_regionkey int NOT NULL, - r_name VARCHAR(25) NOT NULL, - r_comment VARCHAR(152) - )ENGINE=OLAP - DUPLICATE KEY(`r_regionkey`) - COMMENT "OLAP" - DISTRIBUTED BY HASH(`r_regionkey`) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ); - ''' - - sql ''' - drop view if exists revenue0; - ''' - - sql ''' - create view revenue0 (supplier_no, total_revenue) as - select - l_suppkey, - sum(l_extendedprice * (1 - l_discount)) - from - lineitem - where - l_shipdate >= date '1996-01-01' - and l_shipdate < date '1996-01-01' + interval '3' month - group by - l_suppkey; - ''' - - sql ''' - alter table lineitem modify column l_receiptdate set stats ('ndv'='2587', 'avg_size'='16', 'max_size'='16', 'num_nulls'='0', 'min_value'='1992-01-03', 'max_value'='1998-12-31', 'row_count'='3000028242') - ''' - - sql ''' - alter table lineitem modify column l_returnflag set stats ('ndv'='3', 'avg_size'='1', 'max_size'='1', 'num_nulls'='0', 'min_value'='A', 'max_value'='R', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_tax set stats ('ndv'='9', 'avg_size'='16', 'max_size'='16', 'num_nulls'='0', 'min_value'='0', 'max_value'='0.08', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_shipmode set stats ('ndv'='7', 'avg_size'='4', 'max_size'='7', 'num_nulls'='0', 'min_value'='AIR', 'max_value'='TRUCK', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_suppkey set stats ('ndv'='4995648', 'avg_size'='4', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='5000000', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_shipdate set stats ('ndv'='2549', 'avg_size'='16', 'max_size'='16', 'num_nulls'='0', 'min_value'='1992-01-02', 'max_value'='1998-12-01', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_commitdate set stats ('ndv'='2485', 'avg_size'='16', 'max_size'='16', 'num_nulls'='0', 'min_value'='1992-01-31', 'max_value'='1998-10-31', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_partkey set stats ('ndv'='101686464', 'avg_size'='4', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='100000000', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_orderkey set stats ('ndv'='756756416', 'avg_size'='8', 'max_size'='8', 'num_nulls'='0', 'min_value'='1', 'max_value'='3000000000', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_quantity set stats ('ndv'='50', 'avg_size'='16', 'max_size'='16', 'num_nulls'='0', 'min_value'='1', 'max_value'='50', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_linestatus set stats ('ndv'='2', 'avg_size'='1', 'max_size'='1', 'num_nulls'='0', 'min_value'='F', 'max_value'='o', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_comment set stats ('ndv'='154387840', 'avg_size'='26', 'max_size'='43', 'num_nulls'='0', 'min_value'='1', 'max_value'='10000', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_extendedprice set stats ('ndv'='3805461', 'avg_size'='16', 'max_size'='16', 'num_nulls'='0', 'min_value'='900', 'max_value'='104949', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_linenumber set stats ('ndv'='7', 'avg_size'='4', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='7', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_discount set stats ('ndv'='11', 'avg_size'='16', 'max_size'='16', 'num_nulls'='0', 'min_value'='0', 'max_value'='0.1', 'row_count'='3000028242'); - ''' - - sql ''' - alter table lineitem modify column l_shipinstruct set stats ('ndv'='4', 'avg_size'='12', 'max_size'='17', 'num_nulls'='0', 'min_value'='COLLECT COD', 'max_value'='TAKE BACK RETURN', 'row_count'='3000028242'); - ''' - - sql ''' - alter table partsupp modify column ps_suppkey set stats ('ndv'='4995648', 'avg_size'='4.0', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='5000000', 'row_count'='400000000'); - ''' - - sql ''' - alter table partsupp modify column ps_availqty set stats ('ndv'='10008', 'avg_size'='4.0', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='9999', 'row_count'='400000000'); - ''' - - sql ''' - alter table partsupp modify column ps_comment set stats ('ndv'='238546544', 'avg_size'='123.6', 'max_size'='198', 'num_nulls'='0', 'min_value'='Tiresias according to the quiet courts sleep against the ironic', 'max_value'='zzle. unusual decoys detec', 'row_count'='400000000'); - ''' - - sql ''' - alter table partsupp modify column ps_partkey set stats ('ndv'='101686464', 'avg_size'='4.0', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='100000000', 'row_count'='400000000'); - ''' - - sql ''' - alter table partsupp modify column ps_supplycost set stats ('ndv'='100279', 'avg_size'='16.0', 'max_size'='16', 'num_nulls'='0', 'min_value'='1', 'max_value'='1000', 'row_count'='400000000'); - ''' - - sql ''' - alter table supplier modify column s_comment set stats ('ndv'='4942827', 'avg_size'='62.56950', 'max_size'='100', 'num_nulls'='0', 'min_value'='about the blithely express foxes. bli', 'max_value'='zzle furiously. bold accounts haggle fu', 'row_count'='5000000'); - ''' - - sql ''' - alter table supplier modify column s_phone set stats ('ndv'='5070193', 'avg_size'='15.0', 'max_size'='15', 'num_nulls'='0', 'min_value'='10-102-116-6785', 'max_value'='34-998-900-4911', 'row_count'='5000000'); - ''' - - sql ''' - alter table supplier modify column s_nationkey set stats ('ndv'='25', 'avg_size'='4.0', 'max_size'='4', 'num_nulls'='0', 'min_value'='0', 'max_value'='24', 'row_count'='5000000'); - ''' - - sql ''' - alter table supplier modify column s_name set stats ('ndv'='4930031', 'avg_size'='18.0', 'max_size'='18', 'num_nulls'='0', 'min_value'='Supplier#000000001', 'max_value'='Supplier#005000000', 'row_count'='5000000'); - ''' - - sql ''' - alter table supplier modify column s_acctbal set stats ('ndv'='1099069', 'avg_size'='16.0', 'max_size'='16', 'num_nulls'='0', 'min_value'='-998.22', 'max_value'='9999.72', 'row_count'='5000000'); - ''' - - sql ''' - alter table supplier modify column s_address set stats ('ndv'='5000000', 'avg_size'='24.9771', 'max_size'='40', 'num_nulls'='0', 'min_value'=' 9aW1wwnBJJPnCx,n', 'max_value'='zzfDhdtZcvmVzA8rNFU,Yctj1zBN', 'row_count'='5000000'); - ''' - - sql ''' - alter table supplier modify column s_suppkey set stats ('ndv'='5000000', 'avg_size'='4.0', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='5000000', 'row_count'='5000000'); - ''' - - sql ''' - alter table part modify column p_partkey set stats ('ndv'='101686464', 'avg_size'='4.0', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='100000000', 'row_count'='100000000'); - ''' - - sql ''' - alter table part modify column p_container set stats ('ndv'='40', 'avg_size'='7.57', 'max_size'='10', 'num_nulls'='0', 'min_value'='JUMBO BAG', 'max_value'='WRAP PKG', 'row_count'='100000000'); - ''' - - sql ''' - alter table part modify column p_name set stats ('ndv'='98622912', 'avg_size'='32.0', 'max_size'='50', 'num_nulls'='0', 'min_value'='almond antique blue royal burnished', 'max_value'='yellow white seashell lavender black', 'row_count'='100000000'); - ''' - - sql ''' - alter table part modify column p_comment set stats ('ndv'='10190453', 'avg_size'='13', 'max_size'='22', 'num_nulls'='0', 'min_value'='Tire', 'max_value'='zzle. quickly si', 'row_count'='100000000'); - ''' - - sql ''' - alter table part modify column p_brand set stats ('ndv'='25', 'avg_size'='22', 'max_size'='22', 'num_nulls'='0', 'min_value'='Brand#11', 'max_value'='Brand#55', 'row_count'='100000000'); - ''' - - sql ''' - alter table part modify column p_retailprice set stats ('ndv'='120904', 'avg_size'='16.0', 'max_size'='16', 'num_nulls'='0', 'min_value'='901', 'max_value'='2098.99', 'row_count'='100000000'); - ''' - - sql ''' - alter table part modify column p_type set stats ('ndv'='150', 'avg_size'='20', 'max_size'='25', 'num_nulls'='0', 'min_value'='ECONOMY ANODIZED BRASSe', 'max_value'='STANDARD POLISHED TIN ', 'row_count'='100000000'); - ''' - - sql ''' - alter table part modify column p_size set stats ('ndv'='50', 'avg_size'='4', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='50', 'row_count'='100000000'); - ''' - - sql ''' - alter table part modify column p_mfgr set stats ('ndv'='5', 'avg_size'='14', 'max_size'='14', 'num_nulls'='0', 'min_value'='Manufacturer#1', 'max_value'='Manufacturer#5', 'row_count'='100000000'); - ''' - - sql ''' - alter table region modify column r_regionkey set stats ('ndv'='5', 'avg_size'='4', 'max_size'='4', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='5'); - ''' - - sql ''' - alter table region modify column r_name set stats ('ndv'='5', 'avg_size'='14', 'max_size'='14', 'num_nulls'='0', 'min_value'='AFRICA', 'max_value'='MIDDLE EAST', 'row_count'='5'); - ''' - - sql ''' - alter table region modify column r_comment set stats ('ndv'='5', 'avg_size'='14', 'max_size'='14', 'num_nulls'='0', 'min_value'='AFRICA', 'max_value'='MIDDLE EAST', 'row_count'='5'); - ''' - - sql ''' - alter table nation modify column n_nationkey set stats ('ndv'='25', 'avg_size'='4', 'max_size'='4', 'num_nulls'='0', 'min_value'='0', 'max_value'='24', 'row_count'='25'); - ''' - - sql ''' - alter table nation modify column n_regionkey set stats ('ndv'='5', 'avg_size'='4', 'max_size'='4', 'num_nulls'='0', 'min_value'='0', 'max_value'='4', 'row_count'='25'); - ''' - - sql ''' - alter table nation modify column n_name set stats ('ndv'='25', 'avg_size'='14', 'max_size'='14', 'num_nulls'='0', 'min_value'='ALGERIA', 'max_value'='VIETNAM', 'row_count'='25'); - ''' - - sql ''' - alter table nation modify column n_comment set stats ('ndv'='25', 'avg_size'='14', 'max_size'='14', 'num_nulls'='0', 'min_value'='ALGERIA', 'max_value'='VIETNAM', 'row_count'='25'); - ''' - - sql ''' - alter table customer modify column c_custkey set stats ('ndv'='76285424', 'avg_size'='4', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='75000000', 'row_count'='75000000'); - ''' - - sql ''' - alter table customer modify column c_acctbal set stats ('ndv'='1109296', 'avg_size'='16', 'max_size'='16', 'num_nulls'='0', 'min_value'='-999.99', 'max_value'='9999.99', 'row_count'='75000000'); - ''' - - sql ''' - alter table customer modify column c_phone set stats ('ndv'='75783176', 'avg_size'='15', 'max_size'='15', 'num_nulls'='0', 'min_value'='10-100-106-1617', 'max_value'='34-999-618-6881', 'row_count'='75000000'); - ''' - - sql ''' - alter table customer modify column c_mktsegment set stats ('ndv'='5', 'avg_size'='8.9', 'max_size'='10', 'num_nulls'='0', 'min_value'='AUTOMOBILE', 'max_value'='MACHINERY', 'row_count'='75000000'); - ''' - - sql ''' - alter table customer modify column c_address set stats ('ndv'='75000000', 'avg_size'='25', 'max_size'='40', 'num_nulls'='0', 'min_value'=' 2uZwVhQvwA', 'max_value'='zzxGktzXTMKS1BxZlgQ9nqQ', 'row_count'='75000000'); - ''' - - sql ''' - alter table customer modify column c_nationkey set stats ('ndv'='25', 'avg_size'='4', 'max_size'='4', 'num_nulls'='0', 'min_value'='0', 'max_value'='24', 'row_count'='75000000'); - ''' - - sql ''' - alter table customer modify column c_name set stats ('ndv'='75889840', 'avg_size'='25', 'max_size'='25', 'num_nulls'='0', 'min_value'='Customer#000000001', 'max_value'='Customer#075000000', 'row_count'='75000000'); - ''' - - sql ''' - alter table customer modify column c_comment set stats ('ndv'='66629836', 'avg_size'='72', 'max_size'='116', 'num_nulls'='0', 'min_value'=' Tiresias according to the sly', 'max_value'='zzle. blithely regu0', 'row_count'='75000000'); - ''' - - sql ''' - alter table orders modify column o_orderstatus set stats ('ndv'='3', 'avg_size'='1.0', 'max_size'='1', 'num_nulls'='0', 'min_value'='F', 'max_value'='P', 'row_count'='750000000'); - ''' - - sql ''' - alter table orders modify column o_clerk set stats ('ndv'='508347', 'avg_size'='15.0', 'max_size'='15', 'num_nulls'='0', 'min_value'='Clerk#000000001', 'max_value'='Clerk#000500000', 'row_count'='750000000'); - ''' - - sql ''' - alter table orders modify column o_orderdate set stats ('ndv'='2428', 'avg_size'='16.0', 'max_size'='16', 'num_nulls'='0', 'min_value'='1992-01-01', 'max_value'='1998-08-02', 'row_count'='750000000'); - ''' - - sql ''' - alter table orders modify column o_shippriority set stats ('ndv'='1', 'avg_size'='4.0', 'max_size'='4', 'num_nulls'='0', 'min_value'='0', 'max_value'='0', 'row_count'='750000000'); - ''' - - sql ''' - alter table orders modify column o_custkey set stats ('ndv'='50471848', 'avg_size'='4.0', 'max_size'='4', 'num_nulls'='0', 'min_value'='1', 'max_value'='74999999', 'row_count'='750000000'); - ''' - - sql ''' - alter table orders modify column o_totalprice set stats ('ndv'='1462416', 'avg_size'='16.0', 'max_size'='16', 'num_nulls'='0', 'min_value'='857.71', 'max_value'='555285.16', 'row_count'='750000000'); - ''' - - sql ''' - alter table orders modify column o_orderkey set stats ('ndv'='750000000', 'avg_size'='8.0', 'max_size'='8', 'num_nulls'='0', 'min_value'='1', 'max_value'='3000000000', 'row_count'='750000000'); - ''' - - sql ''' - alter table orders modify column o_comment set stats ('ndv'='260882608', 'avg_size'='48.51387', 'max_size'='78', 'num_nulls'='0', 'min_value'='Tiresias about the blithely ironic a', 'max_value'='zzle? furiously ironic instructions among the unusual t ', 'row_count'='750000000'); - ''' - - sql ''' - alter table orders modify column o_orderpriority set stats ('ndv'='5', 'avg_size'='8.4', 'max_size'='15', 'num_nulls'='0', 'min_value'='1-URGENT', 'max_value'='5-LOW', 'row_count'='750000000'); - ''' -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q1.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q1.groovy deleted file mode 100644 index c3770a0ee25a10..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q1.groovy +++ /dev/null @@ -1,56 +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. - */ - -suite("q1") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - sql 'set parallel_pipeline_task_num=8' - -sql 'set be_number_for_test=3' - - - qt_select """ - explain shape plan - select - l_returnflag, - l_linestatus, - sum(l_quantity) as sum_qty, - sum(l_extendedprice) as sum_base_price, - sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, - sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, - avg(l_quantity) as avg_qty, - avg(l_extendedprice) as avg_price, - avg(l_discount) as avg_disc, - count(*) as count_order - from - lineitem - where - l_shipdate <= date '1998-12-01' - interval '90' day - group by - l_returnflag, - l_linestatus - order by - l_returnflag, - l_linestatus; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q10.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q10.groovy deleted file mode 100644 index fdfa09f64c5515..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q10.groovy +++ /dev/null @@ -1,70 +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. - */ - -suite("q10") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - c_custkey, - c_name, - sum(l_extendedprice * (1 - l_discount)) as revenue, - c_acctbal, - n_name, - c_address, - c_phone, - c_comment - from - customer, - orders, - lineitem, - nation - where - c_custkey = o_custkey - and l_orderkey = o_orderkey - and o_orderdate >= date '1993-10-01' - and o_orderdate < date '1993-10-01' + interval '3' month - and l_returnflag = 'R' - and c_nationkey = n_nationkey - group by - c_custkey, - c_name, - c_acctbal, - c_phone, - n_name, - c_address, - c_comment - order by - revenue desc - limit 20; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q11.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q11.groovy deleted file mode 100644 index 034426dca9a4c7..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q11.groovy +++ /dev/null @@ -1,65 +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. - */ - -suite("q11") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - ps_partkey, - sum(ps_supplycost * ps_availqty) as value - from - partsupp, - supplier, - nation - where - ps_suppkey = s_suppkey - and s_nationkey = n_nationkey - and n_name = 'GERMANY' - group by - ps_partkey having - sum(ps_supplycost * ps_availqty) > ( - select - sum(ps_supplycost * ps_availqty) * 0.000002 - from - partsupp, - supplier, - nation - where - ps_suppkey = s_suppkey - and s_nationkey = n_nationkey - and n_name = 'GERMANY' - ) - order by - value desc; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q12.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q12.groovy deleted file mode 100644 index 8827d9a7706137..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q12.groovy +++ /dev/null @@ -1,66 +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. - */ - -suite("q12") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - l_shipmode, - sum(case - when o_orderpriority = '1-URGENT' - or o_orderpriority = '2-HIGH' - then 1 - else 0 - end) as high_line_count, - sum(case - when o_orderpriority <> '1-URGENT' - and o_orderpriority <> '2-HIGH' - then 1 - else 0 - end) as low_line_count - from - orders, - lineitem - where - o_orderkey = l_orderkey - and l_shipmode in ('MAIL', 'SHIP') - and l_commitdate < l_receiptdate - and l_shipdate < l_commitdate - and l_receiptdate >= date '1994-01-01' - and l_receiptdate < date '1994-01-01' + interval '1' year - group by - l_shipmode - order by - l_shipmode; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q13.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q13.groovy deleted file mode 100644 index 994464f783baef..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q13.groovy +++ /dev/null @@ -1,55 +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. - */ - -suite("q13") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - c_count, - count(*) as custdist - from - ( - select - c_custkey, - count(o_orderkey) as c_count - from - customer left outer join orders on - c_custkey = o_custkey - and o_comment not like '%special%requests%' - group by - c_custkey - ) as c_orders - group by - c_count - order by - custdist desc, - c_count desc; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q14.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q14.groovy deleted file mode 100644 index 187e0a0152eabc..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q14.groovy +++ /dev/null @@ -1,48 +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. - */ - -suite("q14") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - 100.00 * sum(case - when p_type like 'PROMO%' - then l_extendedprice * (1 - l_discount) - else 0 - end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue - from - lineitem, - part - where - l_partkey = p_partkey - and l_shipdate >= date '1995-09-01' - and l_shipdate < date '1995-09-01' + interval '1' month; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q15.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q15.groovy deleted file mode 100644 index 4677892fd9abc8..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q15.groovy +++ /dev/null @@ -1,57 +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. - */ - -suite("q15") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - s_suppkey, - s_name, - s_address, - s_phone, - total_revenue - from - supplier, - revenue0 - where - s_suppkey = supplier_no - and total_revenue = ( - select - max(total_revenue) - from - revenue0 - ) - order by - s_suppkey; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q16.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q16.groovy deleted file mode 100644 index 1c3d6f7fce4105..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q16.groovy +++ /dev/null @@ -1,68 +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. - */ - -suite("q16") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - p_brand, - p_type, - p_size, - count(distinct ps_suppkey) as supplier_cnt - from - partsupp, - part - where - p_partkey = ps_partkey - and p_brand <> 'Brand#45' - and p_type not like 'MEDIUM POLISHED%' - and p_size in (49, 14, 23, 45, 19, 3, 36, 9) - and ps_suppkey not in ( - select - s_suppkey - from - supplier - where - s_comment like '%Customer%Complaints%' - ) - group by - p_brand, - p_type, - p_size - order by - supplier_cnt desc, - p_brand, - p_type, - p_size; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q17.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q17.groovy deleted file mode 100644 index 3b79e2a0f5c25f..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q17.groovy +++ /dev/null @@ -1,55 +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. - */ - -suite("q17") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - sum(l_extendedprice) / 7.0 as avg_yearly - from - lineitem, - part - where - p_partkey = l_partkey - and p_brand = 'Brand#23' - and p_container = 'MED BOX' - and l_quantity < ( - select - 0.2 * avg(l_quantity) - from - lineitem - where - l_partkey = p_partkey - ); - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q18.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q18.groovy deleted file mode 100644 index 086ab214e4f27e..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q18.groovy +++ /dev/null @@ -1,71 +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. - */ - -suite("q18") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - c_name, - c_custkey, - o_orderkey, - o_orderdate, - o_totalprice, - sum(l_quantity) - from - customer, - orders, - lineitem - where - o_orderkey in ( - select - l_orderkey - from - lineitem - group by - l_orderkey having - sum(l_quantity) > 300 - ) - and c_custkey = o_custkey - and o_orderkey = l_orderkey - group by - c_name, - c_custkey, - o_orderkey, - o_orderdate, - o_totalprice - order by - o_totalprice desc, - o_orderdate - limit 100; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q19.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q19.groovy deleted file mode 100644 index 00410f506aa108..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q19.groovy +++ /dev/null @@ -1,75 +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. - */ - -suite("q19") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - sum(l_extendedprice* (1 - l_discount)) as revenue - from - lineitem, - part - where - ( - p_partkey = l_partkey - and p_brand = 'Brand#12' - and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') - and l_quantity >= 1 and l_quantity <= 1 + 10 - and p_size between 1 and 5 - and l_shipmode in ('AIR', 'AIR REG') - and l_shipinstruct = 'DELIVER IN PERSON' - ) - or - ( - p_partkey = l_partkey - and p_brand = 'Brand#23' - and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') - and l_quantity >= 10 and l_quantity <= 10 + 10 - and p_size between 1 and 10 - and l_shipmode in ('AIR', 'AIR REG') - and l_shipinstruct = 'DELIVER IN PERSON' - ) - or - ( - p_partkey = l_partkey - and p_brand = 'Brand#34' - and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') - and l_quantity >= 20 and l_quantity <= 20 + 10 - and p_size between 1 and 15 - and l_shipmode in ('AIR', 'AIR REG') - and l_shipinstruct = 'DELIVER IN PERSON' - ); - - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q2.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q2.groovy deleted file mode 100644 index cf0e010b35c9f3..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q2.groovy +++ /dev/null @@ -1,83 +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. - */ - -suite("q2") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - sql 'set parallel_pipeline_task_num=8' - - - -sql 'set be_number_for_test=3' - - - qt_select """ - explain shape plan - select - s_acctbal, - s_name, - n_name, - p_partkey, - p_mfgr, - s_address, - s_phone, - s_comment - from - part, - supplier, - partsupp, - nation, - region - where - p_partkey = ps_partkey - and s_suppkey = ps_suppkey - and p_size = 15 - and p_type like '%BRASS' - and s_nationkey = n_nationkey - and n_regionkey = r_regionkey - and r_name = 'EUROPE' - and ps_supplycost = ( - select - min(ps_supplycost) - from - partsupp, - supplier, - nation, - region - where - p_partkey = ps_partkey - and s_suppkey = ps_suppkey - and s_nationkey = n_nationkey - and n_regionkey = r_regionkey - and r_name = 'EUROPE' - ) - order by - s_acctbal desc, - n_name, - s_name, - p_partkey - limit 100; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q20.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q20.groovy deleted file mode 100644 index b696008b67a435..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q20.groovy +++ /dev/null @@ -1,76 +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. - */ - -suite("q20") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - s_name, - s_address - from - supplier, - nation - where - s_suppkey in ( - select - ps_suppkey - from - partsupp - where - ps_partkey in ( - select - p_partkey - from - part - where - p_name like 'forest%' - ) - and ps_availqty > ( - select - 0.5 * sum(l_quantity) - from - lineitem - where - l_partkey = ps_partkey - and l_suppkey = ps_suppkey - and l_shipdate >= date '1994-01-01' - and l_shipdate < date '1994-01-01' + interval '1' year - ) - ) - and s_nationkey = n_nationkey - and n_name = 'CANADA' - order by - s_name; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q21.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q21.groovy deleted file mode 100644 index 896a2d19d38e21..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q21.groovy +++ /dev/null @@ -1,78 +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. - */ - -suite("q21") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - s_name, - count(*) as numwait - from - supplier, - lineitem l1, - orders, - nation - where - s_suppkey = l1.l_suppkey - and o_orderkey = l1.l_orderkey - and o_orderstatus = 'F' - and l1.l_receiptdate > l1.l_commitdate - and exists ( - select - * - from - lineitem l2 - where - l2.l_orderkey = l1.l_orderkey - and l2.l_suppkey <> l1.l_suppkey - ) - and not exists ( - select - * - from - lineitem l3 - where - l3.l_orderkey = l1.l_orderkey - and l3.l_suppkey <> l1.l_suppkey - and l3.l_receiptdate > l3.l_commitdate - ) - and s_nationkey = n_nationkey - and n_name = 'SAUDI ARABIA' - group by - s_name - order by - numwait desc, - s_name - limit 100; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q22.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q22.groovy deleted file mode 100644 index 5bb8f524610a2d..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q22.groovy +++ /dev/null @@ -1,75 +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. - */ - -suite("q22") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - - - - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - cntrycode, - count(*) as numcust, - sum(c_acctbal) as totacctbal - from - ( - select - substring(c_phone, 1, 2) as cntrycode, - c_acctbal - from - customer - where - substring(c_phone, 1, 2) in - ('13', '31', '23', '29', '30', '18', '17') - and c_acctbal > ( - select - avg(c_acctbal) - from - customer - where - c_acctbal > 0.00 - and substring(c_phone, 1, 2) in - ('13', '31', '23', '29', '30', '18', '17') - ) - and not exists ( - select - * - from - orders - where - o_custkey = c_custkey - ) - ) as custsale - group by - cntrycode - order by - cntrycode; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q3.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q3.groovy deleted file mode 100644 index 45b6a024cb012f..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q3.groovy +++ /dev/null @@ -1,65 +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. - */ - -suite("q3") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - // db = "tpch" - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - sql 'set parallel_pipeline_task_num=8' - - - - - -sql 'set be_number_for_test=3' - - - qt_select """ - explain shape plan - select - l_orderkey, - sum(l_extendedprice * (1 - l_discount)) as revenue, - o_orderdate, - o_shippriority - from - customer, - orders, - lineitem - where - c_mktsegment = 'BUILDING' - and c_custkey = o_custkey - and l_orderkey = o_orderkey - and o_orderdate < date '1995-03-15' - and l_shipdate > date '1995-03-15' - group by - l_orderkey, - o_orderdate, - o_shippriority - order by - revenue desc, - o_orderdate - limit 10; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q4.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q4.groovy deleted file mode 100644 index 26c525509afb7d..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q4.groovy +++ /dev/null @@ -1,61 +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. - */ - -suite("q4") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - sql 'set parallel_pipeline_task_num=8' - - - - -sql 'set be_number_for_test=3' - - - qt_select """ - explain shape plan - select - o_orderpriority, - count(*) as order_count - from - orders - where - o_orderdate >= date '1993-07-01' - and o_orderdate < date '1993-07-01' + interval '3' month - and exists ( - select - * - from - lineitem - where - l_orderkey = o_orderkey - and l_commitdate < l_receiptdate - ) - group by - o_orderpriority - order by - o_orderpriority; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q5.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q5.groovy deleted file mode 100644 index 1cabac98d54961..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q5.groovy +++ /dev/null @@ -1,59 +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. - */ - -suite("q5") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - sql 'set parallel_pipeline_task_num=8' -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - n_name, - sum(l_extendedprice * (1 - l_discount)) as revenue - from - customer, - orders, - lineitem, - supplier, - nation, - region - where - c_custkey = o_custkey - and l_orderkey = o_orderkey - and l_suppkey = s_suppkey - and c_nationkey = s_nationkey - and s_nationkey = n_nationkey - and n_regionkey = r_regionkey - and r_name = 'ASIA' - and o_orderdate >= date '1994-01-01' - and o_orderdate < date '1994-01-01' + interval '1' year - group by - n_name - order by - revenue desc; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q6.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q6.groovy deleted file mode 100644 index 09d3c26e0ce295..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q6.groovy +++ /dev/null @@ -1,44 +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. - */ - -suite("q6") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - - sql 'set global exec_mem_limit = 21G' - sql 'SET enable_pipeline_engine = true' - sql 'set parallel_pipeline_task_num=8' -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - sum(l_extendedprice * l_discount) as revenue - from - lineitem - where - l_shipdate >= date '1994-01-01' - and l_shipdate < date '1994-01-01' + interval '1' year - and l_discount between .06 - 0.01 and .06 + 0.01 - and l_quantity < 24; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q7.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q7.groovy deleted file mode 100644 index fe4bb1341d2f97..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q7.groovy +++ /dev/null @@ -1,74 +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. - */ - -suite("q7") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - sql 'set parallel_pipeline_task_num=8' -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - supp_nation, - cust_nation, - l_year, - sum(volume) as revenue - from - ( - select - n1.n_name as supp_nation, - n2.n_name as cust_nation, - extract(year from l_shipdate) as l_year, - l_extendedprice * (1 - l_discount) as volume - from - supplier, - lineitem, - orders, - customer, - nation n1, - nation n2 - where - s_suppkey = l_suppkey - and o_orderkey = l_orderkey - and c_custkey = o_custkey - and s_nationkey = n1.n_nationkey - and c_nationkey = n2.n_nationkey - and ( - (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY') - or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE') - ) - and l_shipdate between date '1995-01-01' and date '1996-12-31' - ) as shipping - group by - supp_nation, - cust_nation, - l_year - order by - supp_nation, - cust_nation, - l_year; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q8.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q8.groovy deleted file mode 100644 index b70036d34be62f..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q8.groovy +++ /dev/null @@ -1,71 +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. - */ - -suite("q8") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - o_year, - sum(case - when nation = 'BRAZIL' then volume - else 0 - end) / sum(volume) as mkt_share - from - ( - select - extract(year from o_orderdate) as o_year, - l_extendedprice * (1 - l_discount) as volume, - n2.n_name as nation - from - part, - supplier, - lineitem, - orders, - customer, - nation n1, - nation n2, - region - where - p_partkey = l_partkey - and s_suppkey = l_suppkey - and l_orderkey = o_orderkey - and o_custkey = c_custkey - and c_nationkey = n1.n_nationkey - and n1.n_regionkey = r_regionkey - and r_name = 'AMERICA' - and s_nationkey = n2.n_nationkey - and o_orderdate between date '1995-01-01' and date '1996-12-31' - and p_type = 'ECONOMY ANODIZED STEEL' - ) as all_nations - group by - o_year - order by - o_year; - """ -} diff --git a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q9.groovy b/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q9.groovy deleted file mode 100644 index ea5618313ca46a..00000000000000 --- a/regression-test/suites/nereids_tpch_shape_sf500_p0/shape/q9.groovy +++ /dev/null @@ -1,67 +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. - */ - -suite("q9") { - String db = context.config.getDbNameByFile(new File(context.file.parent)) - sql "use ${db}" - sql 'set enable_nereids_planner=true' - sql 'set enable_fallback_to_original_planner=false' - sql "set runtime_filter_mode='GLOBAL'" - sql 'set parallel_pipeline_task_num=8' - sql 'set exec_mem_limit=21G' - sql 'SET enable_pipeline_engine = true' - -sql 'set be_number_for_test=3' - - qt_select """ - explain shape plan - select - nation, - o_year, - sum(amount) as sum_profit - from - ( - select - n_name as nation, - extract(year from o_orderdate) as o_year, - l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount - from - part, - supplier, - lineitem, - partsupp, - orders, - nation - where - s_suppkey = l_suppkey - and ps_suppkey = l_suppkey - and ps_partkey = l_partkey - and p_partkey = l_partkey - and o_orderkey = l_orderkey - and s_nationkey = n_nationkey - and p_name like '%green%' - ) as profit - group by - nation, - o_year - order by - nation, - o_year desc; - """ -} diff --git a/regression-test/suites/partition_p0/multi_partition/test_multi_column_partition.groovy b/regression-test/suites/partition_p0/multi_partition/test_multi_column_partition.groovy index 8ef435cdf17d51..1b78fb30fe32f8 100644 --- a/regression-test/suites/partition_p0/multi_partition/test_multi_column_partition.groovy +++ b/regression-test/suites/partition_p0/multi_partition/test_multi_column_partition.groovy @@ -16,6 +16,10 @@ // under the License. suite("test_multi_partition_key", "p0") { + + // TODO: remove it after we add implicit cast check in Nereids + sql "set enable_nereids_dml=false" + def random = new Random() sql "set enable_insert_strict=true" def createTable = { String tableName, String partitionInfo /* param */ -> diff --git a/regression-test/suites/query_p0/join/test_left_join_null.groovy b/regression-test/suites/query_p0/join/test_left_join_null.groovy index 42227b64852ea4..71efbd87989890 100644 --- a/regression-test/suites/query_p0/join/test_left_join_null.groovy +++ b/regression-test/suites/query_p0/join/test_left_join_null.groovy @@ -20,6 +20,9 @@ suite("test_left_join_null", "query") { def tbName1 = "dept_emp" def tbName2 = "departments" + sql "drop table if exists ${tbName1}" + sql "drop table if exists ${tbName2}" + sql """ CREATE TABLE IF NOT EXISTS ${tbName1} ( `emp_no` int NOT NULL, diff --git a/regression-test/suites/query_p0/union/test_union_pipelineX.groovy b/regression-test/suites/query_p0/union/test_union_pipelineX.groovy deleted file mode 100644 index 861f06b15e9afe..00000000000000 --- a/regression-test/suites/query_p0/union/test_union_pipelineX.groovy +++ /dev/null @@ -1,287 +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. - -suite("test_union_pipelineX") { - def db = "test_query_db" - sql "use ${db}" - -// order_qt_select "select k1, k2 from baseall union select k2, k3 from test" -// order_qt_select "select k2, count(k1) from ((select k2, avg(k1) k1 from baseall group by k2) union all (select k2, count(k1) k1 from test group by k2) )b group by k2 having k2 > 0 order by k2;" - sql """ set experimental_enable_pipeline_x_engine=true; """ - def tbName1 = "test" - def tbName2 = "baseall" - sql """ set experimental_enable_pipeline_x_engine=true; """ - qt_union1 """(select A.k2 as wj1,count(*) as wj2, case A.k2 when 1989 then "wj" - when 1992 then "dyk" when 1985 then "wcx" else "mlx" end - from ${tbName1} as A join ${tbName1} as B where A.k1=B.k1+1 - group by A.k2 having sum(A.k3)> 1989) union all - (select C.k5, C.k8, C.k6 from ${tbName1} as C where lower(C.k6) like "tr%") - order by wj1,wj2""" - qt_union2 """(select A.k2 as wj1,count(*) as wj2, case A.k2 when 1989 then "wj" - when 1992 then "dyk" when 1985 then "wcx" else "mlx" end, - if (A.k2<>255,"hello","world") - from ${tbName1} as A join ${tbName1} as B where A.k1=B.k1+1 - group by A.k2 having sum(A.k3)> 1989) union all - (select C.k5, C.k8, C.k6, if (C.k8<0,"hello","world") - from ${tbName1} as C where lower(C.k6) like "tr%") - order by wj1,wj2""" - qt_union3 """ select A.k2,count(*) from ${tbName1} as A join ${tbName1} as B - where A.k1=B.k1+1 group by A.k2 having sum(A.k3)> 1989 order by A.k2 desc""" - qt_union4 """(select A.k2 as wj1,count(*) as wj2 from ${tbName1} as A join ${tbName1} as B - where A.k1=B.k1+1 group by A.k2 having sum(A.k3)> 1989) - union all (select C.k5, C.k8 from ${tbName2} as C where C.k6 like "tr%") - order by wj1,wj2""" - qt_union5 """(select * from ${tbName1}) union (select * from ${tbName1}) order by k1, k2, k3, k4 limit 4""" - qt_union6 """(select * from ${tbName1}) union all (select * from ${tbName1}) - order by k1, k2, k3, k4 limit 4""" - qt_union7 """(select * from ${tbName1} where k1<10) union all - (select * from ${tbName1} where k5<0) order by k1,k2,k3 limit 40""" - qt_union8 """(select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName1} where k1>0) - union all (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} where k2>0) - union all (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} where k3>0) - order by k1, k2, k3, k4""" - qt_union9 """(select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName1} - where k1>0 order by k1, k2, k3, k4 limit 1) - union all (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} - where k2>0 order by k1, k2, k3, k4 limit 1) - union all (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} - where k3>0 order by k1, k2, k3, k4 limit 1) - order by k1, k2, k3, k4""" - qt_union10 """(select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName1} where k1>0) - union distinct (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} where k2>0) - union distinct (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} where k3>0) - order by k1, k2, k3, k4""" - qt_union11 """(select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName1} where k1>0) - union distinct (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} where k2>0) - union all (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} where k3>0) - order by k1, k2, k3, k4""" - qt_union12 """(select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName1} - where k1>0 order by k1, k2, k3, k4 limit 1) - union all (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} - where k2>0 order by k1, k2, k3, k4 limit 1) - union distinct (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} - where k3>0 order by k1, k2, k3, k4 limit 1) - order by k1, k2, k3, k4""" -// qt_union13 """(select count(k1), sum(k2) from ${tbName1}) -// union all (select k1, k2 from ${tbName1} order by k1, k2 limit 10) -// union all (select sum(k1), max(k3) from ${tbName1} group by k2) -// union all (select k1, k2 from ${tbName2}) -// union all (select a.k1, b.k2 from ${tbName1} a join ${tbName2} b on (a.k1=b.k1)) -// union all (select 1000, 2000) order by k1, k2""" - qt_union14 """select * from (select 1 a, 2 b - union all select 3, 4 - union all select 10, 20) t where a0) - union distinct (select k1, k2, k3, k4, k5, k6, k7, k8, k9, 10, k11 from ${tbName1} where k2>0) - union all (select k1, k2, k3, k4, k5, k6, k7, k8, k9, 10, k11 from ${tbName2} where k3>0)) x""" - qt_union19 """(select 10, 10.0, 'hello', 'world') union all - (select k1, k5, k6, k7 from ${tbName2} where k1=1) union all - (select 20, 20.0, 'wangjuoo4', 'beautiful') union all - (select k2, k8, k6, k7 from ${tbName2} where k2>0) order by 1, 2, 3, 4""" - qt_union20 """select x.k1, k2, k3, k4, k5 from - ((select k1, k2, k3, k4, k5 from ${tbName1} where k1>0) union distinct - (select k1, k2, k3, k4, k5 from ${tbName2} where k2>0)) x - where x.k1<5 and x.k3>0 order by 1, 2, 3, 4""" - qt_union21 """select x.k1, k2, k3, k4, k5 from - ((select k1, k2, k3, k4, k5 from ${tbName1} where k1>0) union all - (select k1, k2, k3, k4, k5 from ${tbName2} where k2>0)) x - where x.k1<5 and x.k3>0 order by 1, 2, 3, 4""" - - - // test_query_union_2 - qt_union22 """select x.k1, k6, k7, k8, k9, k10 from - ((select k1, k6, k7, k8, k9, k10 from ${tbName1} where k1=1) union distinct - (select k1, k6, k7, k8, k9, k10 from ${tbName1} where k9>0)) x union distinct - (select k1, k6, k7, k8, k9, k10 from ${tbName2}) order by 1, 4, 5, 6 limit 10""" - qt_union23 """select x.k1, k6, k7, k8, k9, k10 from - ((select k1, k6, k7, k8, k9, k10 from ${tbName1} where k1=1) union all - (select k1, k6, k7, k8, k9, k10 from ${tbName1} where k9>0)) x union all - (select k1, k6, k7, k8, k9, k10 from ${tbName2}) order by 1, 4, 5, 6 limit 10""" - qt_union24 """(select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName1} where k1>0) - union all (select 1, 2, 3, 4, 3.14, 'hello', 'world', 0.0, 1.1, cast('1989-03-21' as date), - cast('1989-03-21 13:00:00' as datetime)) - union all (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} where k3>0) - order by k1, k2, k3, k4""" - qt_union25 """(select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName1} where k1>0) - union distinct (select 1, 2, 3, 4, 3.14, 'hello', 'world', 0.0, 1.1, cast('1989-03-21' as date), - cast('1989-03-21 13:00:00' as datetime)) - union distinct (select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from ${tbName2} where k3>0) - order by k1, k2, k3, k4""" - - - // test_union_basic - qt_union26 """select 1, 2 union select 1.01, 2.0 union (select 0.0001, 0.0000001) order by 1, 2""" - qt_union27 """select 1, 2 union (select "hell0", "") order by 1, 2""" - qt_union28 """select 1, 2 union select 1.0, 2.0 union (select 1.00000000, 2.00000) order by 1, 2""" - qt_union29 """select 1, 2 union all select 1.0, 2.0 union (select 1.00000000, 2.00000) order by 1, 2""" - qt_union30 """select 1, 2 union all select 1.0, 2.0 union all (select 1.00000000, 2.00000) order by 1, 2""" - qt_union31 """select 1, 2 union select 1.0, 2.0 union all (select 1.00000000, 2.00000) order by 1, 2""" - qt_union32 """select 1, 2 union distinct select 1.0, 2.0 union distinct (select 1.00000000, 2.00000) order by 1, 2""" - qt_union33 """select cast("2016-07-01" as date) union (select "2016-07-02") order by 1""" - qt_union34 """select "2016-07-01" union (select "2016-07-02") order by 1""" - qt_union35 """select cast("2016-07-01" as date) union (select cast("2016-07-02 1:10:0" as date)) order by 1""" - def res1 = sql"""select cast(1 as decimal), cast(2 as double) union distinct select 1.0, 2.0 - union distinct (select 1.00000000, 2.00000) order by 1, 2""" - def res2 = sql"""select cast(1 as decimal), cast(2 as decimal) union distinct select 1.0, 2.0 - union distinct (select 1.00000000, 2.00000) order by 1, 2""" - - // test_union_multi - List sub_sql = ["(select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from baseall where k1 % 3 = 0)"] * 10 - String sql1 = String.join(" union ", sub_sql) + " order by 1, 2, 3, 4" - String sql2 = String.join(" union all ", sub_sql) + " order by 1, 2, 3, 4" - String sql3 = String.join(" union distinct ", sub_sql) + " order by 1, 2, 3, 4" - sql(sql1) - sql(sql1) - sql(sql1) - sql(sql1) - sql(sql1) - sql(sql2) - sql(sql2) - sql(sql2) - sql(sql2) - sql(sql2) - sql(sql3) - sql(sql3) - sql(sql3) - sql(sql3) - sql(sql3) - - - // test_union_bug - // PALO-3617 - qt_union36 """select * from (select 1 as a, 2 as b union select 3, 3) c where a = 1""" - sql """drop view if exists nullable""" - sql """CREATE VIEW `nullable` AS SELECT `a`.`k1` AS `n1`, `b`.`k2` AS `n2` - FROM `default_cluster:${db}`.`baseall` a LEFT OUTER JOIN - `default_cluster:${db}`.`bigtable` b ON `a`.`k1` = `b`.`k1` + 10 - WHERE `b`.`k2` IS NULL""" - order_qt_union37 """select n1 from nullable union all select n2 from nullable""" - qt_union38 """(select n1 from nullable) union all (select n2 from nullable order by n1) order by n1""" - qt_union39 """(select n1 from nullable) union all (select n2 from nullable) order by n1""" - - - // test_union_different_column - // 2个select 的列个数 或 字段类型不相同 - // 列个数会报错;大类型(数值或字符或日期)不同的会报错,大类型相同的成功 - test { - sql "select k1, k2 from ${tbName2} union select k2 from ${tbName1} order by k1, k2" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql "select k1, k1 from ${tbName2} union select k2 from ${tbName1} limit 3" - check { result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql "(select k1, k1 from ${tbName2}) union (select k2, 1 from ${tbName1}) order by k1" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql "(select k1+k1 from ${tbName2}) union (select k2 from ${tbName1}) order by k1+k1" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - - - // 不同类型的列 - def index = 2..10 - index.each { - if (![6, 7, 10].contains(it.toInteger())) { - def res3 = sql"""(select k1 from ${tbName2}) union all (select k${it} from ${tbName1} - order by k${it}) order by k1 limit 30""" - def res4 = sql"""select k1 from ${tbName2} union all (select k${it} from ${tbName1} - order by k${it})order by k1 limit 30""" - check2_doris(res3, res4) - } - } - test { - sql """(select k1, k2 from ${tbName2}) union (select k2, k10 from ${tbName1} order by k10) - order by k1, k2""" - check {result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - // cast类型 - def res5 = sql"""(select k1, k2 from ${tbName2}) union (select k2, cast(k11 as int) from ${tbName1}) - order by k1, k2""" - def res6 = sql"""(select k1, k2 from ${tbName2}) union (select k2, cast(k11 as int) from ${tbName1} order by k2) - order by k1, k2""" - check2_doris(res5, res6) - def res7 = sql"""(select k1, k2 from ${tbName2}) union (select k2, cast(k10 as int) from ${tbName1}) order by k1, k2""" - - def res8 = sql"""(select k1, k2 from ${tbName2}) union (select k2, cast(k10 as int) from ${tbName1} order by k2) order - by k1, k2""" - check2_doris(res7, res8) - // 不同类型不同个数 - test { - sql """select k1, k2 from ${tbName2} union select k11, k10, k9 from ${tbName1} order by k1, k2""" - check {result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - - // test_union_different_schema - def new_union_table = "union_different_schema_table_pipelineX" - sql"""drop table if exists ${new_union_table}""" - sql"""create table if not exists ${new_union_table}(k1 tinyint, k2 decimal(9,3) NULL, k3 char(5) NULL, - k4 date NULL, k5 datetime NULL, - k6 double sum) engine=olap - distributed by hash(k1) buckets 2 properties("storage_type"="column", "replication_num" = "1")""" - //不同schema 不同列报错 - test{ - sql "select * from ${new_union_table} union select * from ${tbName1} order by k1, k2" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - for (idx in range(1, 5)) { - qt_union40 """(select k1 from ${new_union_table}) union (select k${idx} from ${tbName1}) order by k1""" - } - sql"""drop table ${new_union_table}""" - - sql 'set enable_fallback_to_original_planner=false' - sql 'set enable_nereids_planner=true' - qt_union35 """select cast("2016-07-01" as date) union (select cast("2016-07-02 1:10:0" as date)) order by 1""" - - qt_union36 """SELECT a,2 as a FROM (SELECT '1' as a) b where a=1;""" - - test { - sql 'select * from (values (1, 2, 3), (4, 5, 6)) a' - result([[1, 2, 3], [4, 5, 6]]) - } -} diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.groovy index f8369f0ed3beff..013ca819566e34 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_delete_sign.groovy @@ -116,4 +116,72 @@ suite('test_partial_update_delete_sign') { // skip_delete_bitmap=false, skip_delete_sign=true qt_2 "select k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__ from ${tableName2} order by k1,c1,c2,c3,c4,__DORIS_DELETE_SIGN__;" sql "drop table if exists ${tableName2};" + + + // partial update a row that has been deleted by delete sign(table without sequence column) + sql "set skip_delete_sign=false;" + sql "set skip_storage_engine_merge=false;" + sql "set skip_delete_bitmap=false;" + sql "sync" + def tableName3 = "test_partial_update_delete_sign3" + sql "DROP TABLE IF EXISTS ${tableName3};" + sql """ create table ${tableName3} ( + k int, + v1 int, + v2 int + ) ENGINE=OLAP unique key (k) + distributed by hash(k) buckets 1 + properties("replication_num" = "1", + "enable_unique_key_merge_on_write" = "true"); """ + sql "insert into ${tableName3} values(1,1,1);" + qt_1 "select * from ${tableName3} order by k;" + sql "insert into ${tableName3}(k,v1,v2,__DORIS_DELETE_SIGN__) values(1,1,1,1);" + qt_2 "select * from ${tableName3} order by k;" + streamLoad { + table "${tableName3}" + + set 'column_separator', ',' + set 'format', 'csv' + set 'partial_columns', 'true' + set 'columns', 'k,v1' + + file 'test_partial_update_delete_sign_data.csv' + time 10000 // limit inflight 10s + } + sql "sync" + qt_3 "select * from ${tableName3} order by k;" + sql "drop table if exists ${tableName3};" + + + // partial update a row that has been deleted by delete sign(table with sequence column) + def tableName4 = "test_partial_update_delete_sign4" + sql "DROP TABLE IF EXISTS ${tableName4};" + sql """ create table ${tableName4} ( + k int, + v1 int, + v2 int, + c int + ) ENGINE=OLAP unique key (k) + distributed by hash(k) buckets 1 + properties("replication_num" = "1", + "enable_unique_key_merge_on_write" = "true", + "function_column.sequence_col" = "c"); """ + sql "insert into ${tableName4} values(1,1,1,1);" + qt_1 "select * from ${tableName4} order by k;" + sql "insert into ${tableName4}(k,v1,v2,c,__DORIS_DELETE_SIGN__) values(1,1,1,1,1);" + qt_2 "select * from ${tableName4} order by k;" + streamLoad { + table "${tableName4}" + + set 'column_separator', ',' + set 'format', 'csv' + set 'partial_columns', 'true' + set 'columns', 'k,v1' + + file 'test_partial_update_delete_sign_data.csv' + time 10000 // limit inflight 10s + } + sql "sync" + qt_3 "select * from ${tableName4} order by k;" + sql "drop table if exists ${tableName4};" }