diff --git a/be/src/agent/heartbeat_server.cpp b/be/src/agent/heartbeat_server.cpp index 92b121414c156e5..981756efa62ba9d 100644 --- a/be/src/agent/heartbeat_server.cpp +++ b/be/src/agent/heartbeat_server.cpp @@ -62,6 +62,7 @@ void HeartbeatServer::heartbeat(THeartbeatResult& heartbeat_result, << "host:" << master_info.network_address.hostname << ", port:" << master_info.network_address.port << ", cluster id:" << master_info.cluster_id + << ", frontend_info:" << PrintFrontendInfos(master_info.frontend_infos) << ", counter:" << google::COUNTER << ", BE start time: " << _be_epoch; MonotonicStopWatch watch; @@ -92,20 +93,17 @@ Status HeartbeatServer::_heartbeat(const TMasterInfo& master_info) { if (_master_info->cluster_id == -1) { LOG(INFO) << "get first heartbeat. update cluster id"; // write and update cluster id - auto st = _olap_engine->set_cluster_id(master_info.cluster_id); - if (!st.ok()) { - LOG(WARNING) << "fail to set cluster id. status=" << st; - return Status::InternalError("fail to set cluster id."); - } else { - _master_info->cluster_id = master_info.cluster_id; - LOG(INFO) << "record cluster id. host: " << master_info.network_address.hostname - << ". port: " << master_info.network_address.port - << ". cluster id: " << master_info.cluster_id; - } + RETURN_IF_ERROR(_olap_engine->set_cluster_id(master_info.cluster_id)); + + _master_info->cluster_id = master_info.cluster_id; + LOG(INFO) << "record cluster id. host: " << master_info.network_address.hostname + << ". port: " << master_info.network_address.port + << ". cluster id: " << master_info.cluster_id + << ". frontend_infos: " << PrintFrontendInfos(master_info.frontend_infos); } else { if (_master_info->cluster_id != master_info.cluster_id) { - LOG(WARNING) << "invalid cluster id: " << master_info.cluster_id << ". ignore."; - return Status::InternalError("invalid cluster id. ignore."); + return Status::InternalError("invalid cluster id. ignore. cluster_id={}", + master_info.cluster_id); } } @@ -132,10 +130,9 @@ Status HeartbeatServer::_heartbeat(const TMasterInfo& master_info) { std::vector hosts; status = get_hosts(&hosts); if (!status.ok() || hosts.empty()) { - std::stringstream ss; - ss << "the status was not ok when get_hosts, error is " << status.to_string(); - LOG(WARNING) << ss.str(); - return Status::InternalError(ss.str()); + return Status::InternalError( + "the status was not ok when get_hosts, error is {}", + status.to_string()); } //step4: check if the IP of FQDN belongs to the current machine and update BackendOptions._s_localhost @@ -149,12 +146,10 @@ Status HeartbeatServer::_heartbeat(const TMasterInfo& master_info) { } if (!set_new_localhost) { - std::stringstream ss; - ss << "the host recorded in master is " << master_info.backend_ip - << ", but we cannot found the local ip that mapped to that host." - << BackendOptions::get_localhost(); - LOG(WARNING) << ss.str(); - return Status::InternalError(ss.str()); + return Status::InternalError( + "the host recorded in master is {}, but we cannot found the local ip " + "that mapped to that host. backend={}", + master_info.backend_ip, BackendOptions::get_localhost()); } } else { // if is ip,not check anything,use it @@ -179,12 +174,11 @@ Status HeartbeatServer::_heartbeat(const TMasterInfo& master_info) { << ". port: " << _master_info->network_address.port << ". epoch: " << _fe_epoch; } else { - LOG(WARNING) << "epoch is not greater than local. ignore heartbeat. host: " - << _master_info->network_address.hostname - << " port: " << _master_info->network_address.port - << " local epoch: " << _fe_epoch - << " received epoch: " << master_info.epoch; - return Status::InternalError("epoch is not greater than local. ignore heartbeat."); + return Status::InternalError( + "epoch is not greater than local. ignore heartbeat. host: {}, port: {}, local " + "epoch: {}, received epoch: {}", + _master_info->network_address.hostname, _master_info->network_address.port, + _fe_epoch, master_info.epoch); } } else { // when Master FE restarted, host and port remains the same, but epoch will be increased. @@ -200,9 +194,8 @@ Status HeartbeatServer::_heartbeat(const TMasterInfo& master_info) { _master_info->__set_token(master_info.token); LOG(INFO) << "get token. token: " << _master_info->token; } else if (_master_info->token != master_info.token) { - LOG(WARNING) << "invalid token. local_token:" << _master_info->token - << ". token:" << master_info.token; - return Status::InternalError("invalid token."); + return Status::InternalError("invalid token. local_token: {}, token: {}", + _master_info->token, master_info.token); } } @@ -219,6 +212,10 @@ Status HeartbeatServer::_heartbeat(const TMasterInfo& master_info) { _master_info->__set_backend_id(master_info.backend_id); } + if (master_info.__isset.frontend_infos) { + ExecEnv::GetInstance()->update_frontends(master_info.frontend_infos); + } + if (need_report) { LOG(INFO) << "Master FE is changed or restarted. report tablet and disk info immediately"; _olap_engine->notify_listeners(); @@ -228,8 +225,8 @@ Status HeartbeatServer::_heartbeat(const TMasterInfo& master_info) { } Status create_heartbeat_server(ExecEnv* exec_env, uint32_t server_port, - ThriftServer** thrift_server, uint32_t worker_thread_num, - TMasterInfo* local_master_info) { + std::unique_ptr* thrift_server, + uint32_t worker_thread_num, TMasterInfo* local_master_info) { HeartbeatServer* heartbeat_server = new HeartbeatServer(local_master_info); if (heartbeat_server == nullptr) { return Status::InternalError("Get heartbeat server failed"); @@ -240,8 +237,8 @@ Status create_heartbeat_server(ExecEnv* exec_env, uint32_t server_port, std::shared_ptr handler(heartbeat_server); std::shared_ptr server_processor( new HeartbeatServiceProcessor(handler)); - *thrift_server = - new ThriftServer("heartbeat", server_processor, server_port, worker_thread_num); + *thrift_server = std::make_unique("heartbeat", server_processor, server_port, + worker_thread_num); return Status::OK(); } } // namespace doris diff --git a/be/src/agent/heartbeat_server.h b/be/src/agent/heartbeat_server.h index e08603043f31879..aee053860a0ec0a 100644 --- a/be/src/agent/heartbeat_server.h +++ b/be/src/agent/heartbeat_server.h @@ -66,6 +66,6 @@ class HeartbeatServer : public HeartbeatServiceIf { }; // class HeartBeatServer Status create_heartbeat_server(ExecEnv* exec_env, uint32_t heartbeat_server_port, - ThriftServer** heart_beat_server, uint32_t worker_thread_num, - TMasterInfo* local_master_info); + std::unique_ptr* heart_beat_server, + uint32_t worker_thread_num, TMasterInfo* local_master_info); } // namespace doris diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 9c2df268a7ae1b6..71fe09b9d91e24f 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -355,7 +355,7 @@ void TaskWorkerPool::_alter_inverted_index_worker_thread_callback() { alter_inverted_index_rq.tablet_id); if (tablet_ptr != nullptr) { EngineIndexChangeTask engine_task(alter_inverted_index_rq); - status = _env->storage_engine()->execute_task(&engine_task); + status = StorageEngine::instance()->execute_task(&engine_task); } else { status = Status::NotFound("could not find tablet {}", alter_inverted_index_rq.tablet_id); @@ -570,7 +570,7 @@ void TaskWorkerPool::_check_consistency_worker_thread_callback() { EngineChecksumTask engine_task(check_consistency_req.tablet_id, check_consistency_req.schema_hash, check_consistency_req.version, &checksum); - Status status = _env->storage_engine()->execute_task(&engine_task); + Status status = StorageEngine::instance()->execute_task(&engine_task); if (!status.ok()) { LOG_WARNING("failed to check consistency") .tag("signature", agent_task_req.signature) @@ -667,7 +667,7 @@ void TaskWorkerPool::_report_disk_state_worker_thread_callback() { request.__isset.disks = true; std::vector data_dir_infos; - _env->storage_engine()->get_all_data_dir_info(&data_dir_infos, true /* update */); + StorageEngine::instance()->get_all_data_dir_info(&data_dir_infos, true /* update */); for (auto& root_path_info : data_dir_infos) { TDisk disk; @@ -1305,7 +1305,7 @@ void CreateTableTaskPool::_create_tablet_worker_thread_callback() { std::vector finish_tablet_infos; VLOG_NOTICE << "create tablet: " << create_tablet_req; - Status status = _env->storage_engine()->create_tablet(create_tablet_req, profile); + Status status = StorageEngine::instance()->create_tablet(create_tablet_req, profile); if (!status.ok()) { DorisMetrics::instance()->create_tablet_requests_failed->increment(1); LOG_WARNING("failed to create tablet, reason={}", status.to_string()) @@ -1384,7 +1384,7 @@ void DropTableTaskPool::_drop_tablet_worker_thread_callback() { // if tablet is dropped by fe, then the related txn should also be removed StorageEngine::instance()->txn_manager()->force_rollback_tablet_related_txns( dropped_tablet->data_dir()->get_meta(), drop_tablet_req.tablet_id, - drop_tablet_req.schema_hash, dropped_tablet->tablet_uid()); + dropped_tablet->tablet_uid()); LOG_INFO("successfully drop tablet") .tag("signature", agent_task_req.signature) .tag("tablet_id", drop_tablet_req.tablet_id); @@ -1470,7 +1470,7 @@ void PushTaskPool::_push_worker_thread_callback() { std::vector tablet_infos; EngineBatchLoadTask engine_task(push_req, &tablet_infos); - auto status = _env->storage_engine()->execute_task(&engine_task); + auto status = StorageEngine::instance()->execute_task(&engine_task); // Return result to fe TFinishTaskRequest finish_task_request; @@ -1539,7 +1539,7 @@ void PublishVersionTaskPool::_publish_version_worker_thread_callback() { error_tablet_ids.clear(); EnginePublishVersionTask engine_task(publish_version_req, &error_tablet_ids, &succ_tablet_ids, &discontinuous_version_tablets); - status = _env->storage_engine()->execute_task(&engine_task); + status = StorageEngine::instance()->execute_task(&engine_task); if (status.ok()) { break; } else if (status.is()) { @@ -1664,11 +1664,11 @@ void ClearTransactionTaskPool::_clear_transaction_task_worker_thread_callback() // If it is not greater than zero, no need to execute // the following clear_transaction_task() function. if (!clear_transaction_task_req.partition_id.empty()) { - _env->storage_engine()->clear_transaction_task( + StorageEngine::instance()->clear_transaction_task( clear_transaction_task_req.transaction_id, clear_transaction_task_req.partition_id); } else { - _env->storage_engine()->clear_transaction_task( + StorageEngine::instance()->clear_transaction_task( clear_transaction_task_req.transaction_id); } LOG(INFO) << "finish to clear transaction task. signature=" << agent_task_req.signature @@ -1765,7 +1765,7 @@ void AlterTableTaskPool::_alter_tablet(const TAgentTaskRequest& agent_task_req, new_tablet_id = agent_task_req.alter_tablet_req_v2.new_tablet_id; new_schema_hash = agent_task_req.alter_tablet_req_v2.new_schema_hash; EngineAlterTabletTask engine_task(agent_task_req.alter_tablet_req_v2); - status = _env->storage_engine()->execute_task(&engine_task); + status = StorageEngine::instance()->execute_task(&engine_task); } if (status.ok()) { @@ -1866,7 +1866,7 @@ void CloneTaskPool::_clone_worker_thread_callback() { std::vector tablet_infos; EngineCloneTask engine_task(clone_req, _master_info, agent_task_req.signature, &tablet_infos); - auto status = _env->storage_engine()->execute_task(&engine_task); + auto status = StorageEngine::instance()->execute_task(&engine_task); // Return result to fe TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(BackendOptions::get_local_backend()); @@ -1923,7 +1923,7 @@ void StorageMediumMigrateTaskPool::_storage_medium_migrate_worker_thread_callbac auto status = _check_migrate_request(storage_medium_migrate_req, tablet, &dest_store); if (status.ok()) { EngineStorageMigrationTask engine_task(tablet, dest_store); - status = _env->storage_engine()->execute_task(&engine_task); + status = StorageEngine::instance()->execute_task(&engine_task); } if (!status.ok()) { LOG_WARNING("failed to migrate storage medium") diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 7e0ee4e536abc8c..558d35b522f06f2 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -286,6 +286,9 @@ DEFINE_mInt32(default_num_rows_per_column_file_block, "1024"); DEFINE_mInt32(pending_data_expire_time_sec, "1800"); // inc_rowset snapshot rs sweep time interval DEFINE_mInt32(tablet_rowset_stale_sweep_time_sec, "300"); +// tablet stale rowset sweep by threshold size +DEFINE_Bool(tablet_rowset_stale_sweep_by_size, "false"); +DEFINE_mInt32(tablet_rowset_stale_sweep_threshold_size, "100"); // garbage sweep policy DEFINE_Int32(max_garbage_sweep_interval, "3600"); DEFINE_Int32(min_garbage_sweep_interval, "180"); @@ -297,6 +300,7 @@ DEFINE_mInt32(trash_file_expire_time_sec, "259200"); // minimum file descriptor number // modify them upon necessity DEFINE_Int32(min_file_descriptor_number, "60000"); +DEFINE_mBool(disable_segment_cache, "false"); DEFINE_Int64(index_stream_cache_capacity, "10737418240"); DEFINE_String(row_cache_mem_limit, "20%"); @@ -483,6 +487,8 @@ DEFINE_mInt32(stream_load_record_batch_size, "50"); DEFINE_Int32(stream_load_record_expire_time_secs, "28800"); // time interval to clean expired stream load records DEFINE_mInt64(clean_stream_load_record_interval_secs, "1800"); +// The buffer size to store stream table function schema info +DEFINE_Int64(stream_tvf_buffer_size, "1048576"); // 1MB // OlapTableSink sender's send interval, should be less than the real response time of a tablet writer rpc. // You may need to lower the speed when the sink receiver bes are too busy. @@ -845,6 +851,9 @@ DEFINE_Validator(jsonb_type_length_soft_limit_bytes, // is greater than object_pool_buffer_size, release the object in the unused_object_pool. DEFINE_Int32(object_pool_buffer_size, "100"); +// Threshold of reading a small file into memory +DEFINE_mInt32(in_memory_file_size, "1048576"); // 1MB + // ParquetReaderWrap prefetch buffer size DEFINE_Int32(parquet_reader_max_buffer_size, "50"); // Max size of parquet page header in bytes @@ -1070,6 +1079,8 @@ DEFINE_mString(user_files_secure_path, "${DORIS_HOME}"); DEFINE_Int32(partition_topn_partition_threshold, "1024"); +DEFINE_Int32(fe_expire_duration_seconds, "60"); + #ifdef BE_TEST // test s3 DEFINE_String(test_s3_resource, "resource"); @@ -1507,7 +1518,11 @@ std::vector> get_config_info() { _config.push_back(it.first); _config.push_back(field_it->second.type); - _config.push_back(it.second); + if (0 == strcmp(field_it->second.type, "bool")) { + _config.push_back(it.second == "1" ? "true" : "false"); + } else { + _config.push_back(it.second); + } _config.push_back(field_it->second.valmutable ? "true" : "false"); configs.push_back(_config); diff --git a/be/src/common/config.h b/be/src/common/config.h index 89aa6d4e14432db..5c098b0806d347f 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -332,6 +332,9 @@ DECLARE_mInt32(default_num_rows_per_column_file_block); DECLARE_mInt32(pending_data_expire_time_sec); // inc_rowset snapshot rs sweep time interval DECLARE_mInt32(tablet_rowset_stale_sweep_time_sec); +// tablet stale rowset sweep by threshold size +DECLARE_Bool(tablet_rowset_stale_sweep_by_size); +DECLARE_mInt32(tablet_rowset_stale_sweep_threshold_size); // garbage sweep policy DECLARE_Int32(max_garbage_sweep_interval); DECLARE_Int32(min_garbage_sweep_interval); @@ -344,6 +347,7 @@ DECLARE_mInt32(trash_file_expire_time_sec); // minimum file descriptor number // modify them upon necessity DECLARE_Int32(min_file_descriptor_number); +DECLARE_mBool(disable_segment_cache); DECLARE_Int64(index_stream_cache_capacity); DECLARE_String(row_cache_mem_limit); @@ -356,6 +360,7 @@ DECLARE_Int32(storage_page_cache_shard_size); // all storage page cache will be divided into data_page_cache and index_page_cache DECLARE_Int32(index_page_cache_percentage); // whether to disable page cache feature in storage +// TODO delete it. Divided into Data page, Index page, pk index page DECLARE_Bool(disable_storage_page_cache); // whether to disable row cache feature in storage DECLARE_Bool(disable_storage_row_cache); @@ -534,6 +539,8 @@ DECLARE_mInt32(stream_load_record_batch_size); DECLARE_Int32(stream_load_record_expire_time_secs); // time interval to clean expired stream load records DECLARE_mInt64(clean_stream_load_record_interval_secs); +// The buffer size to store stream table function schema info +DECLARE_Int64(stream_tvf_buffer_size); // OlapTableSink sender's send interval, should be less than the real response time of a tablet writer rpc. // You may need to lower the speed when the sink receiver bes are too busy. @@ -892,6 +899,9 @@ DECLARE_mInt32(jsonb_type_length_soft_limit_bytes); // is greater than object_pool_buffer_size, release the object in the unused_object_pool. DECLARE_Int32(object_pool_buffer_size); +// Threshold fo reading a small file into memory +DECLARE_mInt32(in_memory_file_size); + // ParquetReaderWrap prefetch buffer size DECLARE_Int32(parquet_reader_max_buffer_size); // Max size of parquet page header in bytes @@ -1129,6 +1139,10 @@ DECLARE_mString(user_files_secure_path); // and if this threshold is exceeded, the remaining data will be pass through to other node directly. DECLARE_Int32(partition_topn_partition_threshold); +// If fe's frontend info has not been updated for more than fe_expire_duration_seconds, it will be regarded +// as an abnormal fe, this will cause be to cancel this fe's related query. +DECLARE_Int32(fe_expire_duration_seconds); + #ifdef BE_TEST // test s3 DECLARE_String(test_s3_resource); diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp index fe971165bd9701d..c17bf1367f01ef8 100644 --- a/be/src/common/daemon.cpp +++ b/be/src/common/daemon.cpp @@ -49,8 +49,6 @@ #include "runtime/memory/mem_tracker.h" #include "runtime/memory/mem_tracker_limiter.h" #include "runtime/task_group/task_group_manager.h" -#include "runtime/user_function_cache.h" -#include "service/backend_options.h" #include "util/cpu_info.h" #include "util/debug_util.h" #include "util/disk_info.h" @@ -65,6 +63,10 @@ namespace doris { +Daemon::~Daemon() { + stop(); +} + void Daemon::tcmalloc_gc_thread() { // TODO All cache GC wish to be supported #if !defined(ADDRESS_SANITIZER) && !defined(LEAK_SANITIZER) && !defined(THREAD_SANITIZER) && \ @@ -187,11 +189,7 @@ void Daemon::memory_maintenance_thread() { int32_t interval_milliseconds = config::memory_maintenance_sleep_time_ms; int64_t last_print_proc_mem = PerfCounters::get_vm_rss(); while (!_stop_background_threads_latch.wait_for( - std::chrono::milliseconds(interval_milliseconds)) && - !k_doris_exit) { - if (!MemInfo::initialized() || !ExecEnv::GetInstance()->initialized()) { - continue; - } + std::chrono::milliseconds(interval_milliseconds))) { // Refresh process memory metrics. doris::PerfCounters::refresh_proc_status(); doris::MemInfo::refresh_proc_meminfo(); @@ -227,10 +225,8 @@ void Daemon::memory_gc_thread() { int32_t memory_full_gc_sleep_time_ms = 0; int32_t memory_gc_sleep_time_ms = config::memory_gc_sleep_time_ms; while (!_stop_background_threads_latch.wait_for( - std::chrono::milliseconds(interval_milliseconds)) && - !k_doris_exit) { - if (config::disable_memory_gc || !MemInfo::initialized() || - !ExecEnv::GetInstance()->initialized()) { + std::chrono::milliseconds(interval_milliseconds))) { + if (config::disable_memory_gc) { continue; } auto sys_mem_available = doris::MemInfo::sys_mem_available(); @@ -280,11 +276,8 @@ void Daemon::memtable_memory_limiter_tracker_refresh_thread() { // Refresh the memory statistics of the load channel tracker more frequently, // which helps to accurately control the memory of LoadChannelMgr. while (!_stop_background_threads_latch.wait_for( - std::chrono::milliseconds(config::memtable_mem_tracker_refresh_interval_ms)) && - !k_doris_exit) { - if (ExecEnv::GetInstance()->initialized()) { - doris::ExecEnv::GetInstance()->memtable_memory_limiter()->refresh_mem_tracker(); - } + std::chrono::milliseconds(config::memtable_mem_tracker_refresh_interval_ms))) { + doris::ExecEnv::GetInstance()->memtable_memory_limiter()->refresh_mem_tracker(); } } @@ -305,9 +298,6 @@ void Daemon::calculate_metrics_thread() { std::map lst_net_receive_bytes; do { - if (!ExecEnv::GetInstance()->initialized()) { - continue; - } DorisMetrics::instance()->metric_registry()->trigger_all_hooks(true); if (last_ts == -1L) { @@ -356,146 +346,57 @@ void Daemon::calculate_metrics_thread() { DorisMetrics::instance()->all_segments_num->set_value( StorageEngine::instance()->tablet_manager()->get_segment_nums()); } - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(15)) && !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(15))); } // clean up stale spilled files void Daemon::block_spill_gc_thread() { - while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(60)) && !k_doris_exit) { - if (ExecEnv::GetInstance()->initialized()) { - ExecEnv::GetInstance()->block_spill_mgr()->gc(200); - } - } -} - -static void init_doris_metrics(const std::vector& store_paths) { - bool init_system_metrics = config::enable_system_metrics; - std::set disk_devices; - std::vector network_interfaces; - std::vector paths; - for (auto& store_path : store_paths) { - paths.emplace_back(store_path.path); - } - if (init_system_metrics) { - auto st = DiskInfo::get_disk_devices(paths, &disk_devices); - if (!st.ok()) { - LOG(WARNING) << "get disk devices failed, status=" << st; - return; - } - st = get_inet_interfaces(&network_interfaces, BackendOptions::is_bind_ipv6()); - if (!st.ok()) { - LOG(WARNING) << "get inet interfaces failed, status=" << st; - return; - } - } - DorisMetrics::instance()->initialize(init_system_metrics, disk_devices, network_interfaces); -} - -void signal_handler(int signal) { - if (signal == SIGINT || signal == SIGTERM) { - k_doris_exit = true; - } -} - -int install_signal(int signo, void (*handler)(int)) { - struct sigaction sa; - memset(&sa, 0, sizeof(struct sigaction)); - sa.sa_handler = handler; - sigemptyset(&sa.sa_mask); - auto ret = sigaction(signo, &sa, nullptr); - if (ret != 0) { - char buf[64]; - LOG(ERROR) << "install signal failed, signo=" << signo << ", errno=" << errno - << ", errmsg=" << strerror_r(errno, buf, sizeof(buf)); - } - return ret; -} - -void init_signals() { - auto ret = install_signal(SIGINT, signal_handler); - if (ret < 0) { - exit(-1); - } - ret = install_signal(SIGTERM, signal_handler); - if (ret < 0) { - exit(-1); + while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(60))) { + ExecEnv::GetInstance()->block_spill_mgr()->gc(200); } } -void Daemon::init(int argc, char** argv, const std::vector& paths) { - // google::SetVersionString(get_build_version(false)); - // google::ParseCommandLineFlags(&argc, &argv, true); - google::ParseCommandLineFlags(&argc, &argv, true); - init_glog("be"); - - LOG(INFO) << get_version_string(false); - - init_thrift_logging(); - CpuInfo::init(); - DiskInfo::init(); - MemInfo::init(); - UserFunctionCache::instance()->init(config::user_function_dir); - - LOG(INFO) << CpuInfo::debug_string(); - LOG(INFO) << DiskInfo::debug_string(); - LOG(INFO) << MemInfo::debug_string(); - - init_doris_metrics(paths); - init_signals(); -} - void Daemon::start() { Status st; st = Thread::create( "Daemon", "tcmalloc_gc_thread", [this]() { this->tcmalloc_gc_thread(); }, - &_tcmalloc_gc_thread); + &_threads.emplace_back()); CHECK(st.ok()) << st; st = Thread::create( "Daemon", "memory_maintenance_thread", [this]() { this->memory_maintenance_thread(); }, - &_memory_maintenance_thread); + &_threads.emplace_back()); CHECK(st.ok()) << st; st = Thread::create( "Daemon", "memory_gc_thread", [this]() { this->memory_gc_thread(); }, - &_memory_gc_thread); + &_threads.emplace_back()); CHECK(st.ok()) << st; st = Thread::create( "Daemon", "memtable_memory_limiter_tracker_refresh_thread", [this]() { this->memtable_memory_limiter_tracker_refresh_thread(); }, - &_memtable_memory_limiter_tracker_refresh_thread); + &_threads.emplace_back()); CHECK(st.ok()) << st; if (config::enable_metric_calculator) { st = Thread::create( "Daemon", "calculate_metrics_thread", - [this]() { this->calculate_metrics_thread(); }, &_calculate_metrics_thread); + [this]() { this->calculate_metrics_thread(); }, &_threads.emplace_back()); CHECK(st.ok()) << st; } st = Thread::create( "Daemon", "block_spill_gc_thread", [this]() { this->block_spill_gc_thread(); }, - &_block_spill_gc_thread); + &_threads.emplace_back()); CHECK(st.ok()) << st; } void Daemon::stop() { - _stop_background_threads_latch.count_down(); - - if (_tcmalloc_gc_thread) { - _tcmalloc_gc_thread->join(); - } - if (_memory_maintenance_thread) { - _memory_maintenance_thread->join(); - } - if (_memory_gc_thread) { - _memory_gc_thread->join(); + if (_stop_background_threads_latch.count() == 0) { + return; } - if (_memtable_memory_limiter_tracker_refresh_thread) { - _memtable_memory_limiter_tracker_refresh_thread->join(); - } - if (_calculate_metrics_thread) { - _calculate_metrics_thread->join(); - } - if (_block_spill_gc_thread) { - _block_spill_gc_thread->join(); + _stop_background_threads_latch.count_down(); + for (auto&& t : _threads) { + if (t) { + t->join(); + } } } diff --git a/be/src/common/daemon.h b/be/src/common/daemon.h index 8ea4c08e48ae08e..c552f55f03198f3 100644 --- a/be/src/common/daemon.h +++ b/be/src/common/daemon.h @@ -25,17 +25,10 @@ namespace doris { -struct StorePath; -inline bool k_doris_exit = false; - class Daemon { public: Daemon() : _stop_background_threads_latch(1) {} - - // Initialises logging, flags etc. Callers that want to override default gflags - // variables should do so before calling this method; no logging should be - // performed until after this method returns. - void init(int argc, char** argv, const std::vector& paths); + ~Daemon(); // Start background threads void start(); @@ -52,11 +45,6 @@ class Daemon { void block_spill_gc_thread(); CountDownLatch _stop_background_threads_latch; - scoped_refptr _tcmalloc_gc_thread; - scoped_refptr _memory_maintenance_thread; - scoped_refptr _memory_gc_thread; - scoped_refptr _memtable_memory_limiter_tracker_refresh_thread; - scoped_refptr _calculate_metrics_thread; - scoped_refptr _block_spill_gc_thread; + std::vector> _threads; }; } // namespace doris diff --git a/be/src/common/stack_trace.cpp b/be/src/common/stack_trace.cpp index f5fb2a19014c147..703dfe49b67202f 100644 --- a/be/src/common/stack_trace.cpp +++ b/be/src/common/stack_trace.cpp @@ -299,7 +299,7 @@ StackTrace::StackTrace(const ucontext_t& signal_context) { void StackTrace::tryCapture() { // When unw_backtrace is not available, fall back on the standard // `backtrace` function from execinfo.h. -#if USE_UNWIND +#if USE_UNWIND && defined(__x86_64__) // TODO size = unw_backtrace(frame_pointers.data(), capacity); #else size = backtrace(frame_pointers.data(), capacity); diff --git a/be/src/common/status.h b/be/src/common/status.h index 02b8e79053e5a39..ddc3344610cb2b6 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -268,12 +268,14 @@ E(INVERTED_INDEX_NO_TERMS, -6005); E(INVERTED_INDEX_RENAME_FILE_FAILED, -6006); E(INVERTED_INDEX_EVALUATE_SKIPPED, -6007); E(INVERTED_INDEX_BUILD_WAITTING, -6008); +E(KEY_NOT_FOUND, -6009); +E(KEY_ALREADY_EXISTS, -6010); #undef E } // namespace ErrorCode // clang-format off // whether to capture stacktrace -consteval bool capture_stacktrace(int code) { +constexpr bool capture_stacktrace(int code) { return code != ErrorCode::OK && code != ErrorCode::END_OF_FILE && code != ErrorCode::MEM_LIMIT_EXCEEDED @@ -305,7 +307,11 @@ consteval bool capture_stacktrace(int code) { && code != ErrorCode::TRANSACTION_NOT_EXIST && code != ErrorCode::TRANSACTION_ALREADY_VISIBLE && code != ErrorCode::TOO_MANY_TRANSACTIONS - && code != ErrorCode::TRANSACTION_ALREADY_COMMITTED; + && code != ErrorCode::TRANSACTION_ALREADY_COMMITTED + && code != ErrorCode::KEY_NOT_FOUND + && code != ErrorCode::KEY_ALREADY_EXISTS + && code != ErrorCode::CANCELLED + && code != ErrorCode::UNINITIALIZED; } // clang-format on @@ -377,7 +383,7 @@ class Status { status._err_msg->_msg = fmt::format(msg, std::forward(args)...); } #ifdef ENABLE_STACKTRACE - if constexpr (stacktrace) { + if (stacktrace && capture_stacktrace(code)) { status._err_msg->_stack = get_stack_trace(); LOG(WARNING) << "meet error status: " << status; // may print too many stacks. } @@ -387,10 +393,10 @@ class Status { static Status OK() { return Status(); } -#define ERROR_CTOR(name, code) \ - template \ - static Status name(std::string_view msg, Args&&... args) { \ - return Error(msg, std::forward(args)...); \ +#define ERROR_CTOR(name, code) \ + template \ + static Status name(std::string_view msg, Args&&... args) { \ + return Error(msg, std::forward(args)...); \ } ERROR_CTOR(PublishTimeout, PUBLISH_TIMEOUT) diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp index 53f44dd590b2906..5c40475eeb3704c 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -30,12 +30,12 @@ #include #include "common/config.h" +#include "vec/sink/async_writer_sink.h" #include "vec/sink/multi_cast_data_stream_sink.h" #include "vec/sink/vdata_stream_sender.h" #include "vec/sink/vmemory_scratch_sink.h" #include "vec/sink/vresult_file_sink.h" #include "vec/sink/vresult_sink.h" -#include "vec/sink/vtable_sink.h" #include "vec/sink/vtablet_sink.h" #include "vec/sink/vtablet_sink_v2.h" @@ -92,9 +92,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink params.destinations, send_query_statistics_with_every_batch, output_exprs, desc_tbl)); } else { - sink->reset(new doris::vectorized::VResultFileSink( - state, pool, row_desc, thrift_sink.result_file_sink, - send_query_statistics_with_every_batch, output_exprs)); + sink->reset(new doris::vectorized::VResultFileSink(row_desc, output_exprs)); } break; } @@ -112,7 +110,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink return Status::InternalError("Missing data buffer sink."); } vectorized::VMysqlTableSink* vmysql_tbl_sink = - new vectorized::VMysqlTableSink(pool, row_desc, output_exprs); + new vectorized::VMysqlTableSink(row_desc, output_exprs); sink->reset(vmysql_tbl_sink); break; #else @@ -124,7 +122,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink if (!thrift_sink.__isset.odbc_table_sink) { return Status::InternalError("Missing data odbc sink."); } - sink->reset(new vectorized::VOdbcTableSink(pool, row_desc, output_exprs)); + sink->reset(new vectorized::VOdbcTableSink(row_desc, output_exprs)); break; } @@ -133,7 +131,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink return Status::InternalError("Missing data jdbc sink."); } if (config::enable_java_support) { - sink->reset(new vectorized::VJdbcTableSink(pool, row_desc, output_exprs)); + sink->reset(new vectorized::VJdbcTableSink(row_desc, output_exprs)); } else { return Status::InternalError( "Jdbc table sink is not enabled, you can change be config " @@ -234,9 +232,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink params.destinations, send_query_statistics_with_every_batch, output_exprs, desc_tbl)); } else { - sink->reset(new doris::vectorized::VResultFileSink( - state, pool, row_desc, thrift_sink.result_file_sink, - send_query_statistics_with_every_batch, output_exprs)); + sink->reset(new doris::vectorized::VResultFileSink(row_desc, output_exprs)); } break; } @@ -254,7 +250,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink return Status::InternalError("Missing data buffer sink."); } vectorized::VMysqlTableSink* vmysql_tbl_sink = - new vectorized::VMysqlTableSink(pool, row_desc, output_exprs); + new vectorized::VMysqlTableSink(row_desc, output_exprs); sink->reset(vmysql_tbl_sink); break; #else @@ -266,7 +262,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink if (!thrift_sink.__isset.odbc_table_sink) { return Status::InternalError("Missing data odbc sink."); } - sink->reset(new vectorized::VOdbcTableSink(pool, row_desc, output_exprs)); + sink->reset(new vectorized::VOdbcTableSink(row_desc, output_exprs)); break; } @@ -275,7 +271,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink return Status::InternalError("Missing data jdbc sink."); } if (config::enable_java_support) { - sink->reset(new vectorized::VJdbcTableSink(pool, row_desc, output_exprs)); + sink->reset(new vectorized::VJdbcTableSink(row_desc, output_exprs)); } else { return Status::InternalError( "Jdbc table sink is not enabled, you can change be config " diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h index 8200ff5d4c1dd99..778ee2da83adce5 100644 --- a/be/src/exec/data_sink.h +++ b/be/src/exec/data_sink.h @@ -104,7 +104,7 @@ class DataSink { DescriptorTbl& desc_tbl); // Returns the runtime profile for the sink. - virtual RuntimeProfile* profile() = 0; + RuntimeProfile* profile() { return _profile; } virtual void set_query_statistics(std::shared_ptr statistics) { _query_statistics = statistics; @@ -121,6 +121,8 @@ class DataSink { std::string _name; const RowDescriptor& _row_desc; + RuntimeProfile* _profile = nullptr; // Allocated from _pool + // Maybe this will be transferred to BufferControlBlock. std::shared_ptr _query_statistics; diff --git a/be/src/exec/rowid_fetcher.cpp b/be/src/exec/rowid_fetcher.cpp index af8e71eed006260..302fc34fcc583f3 100644 --- a/be/src/exec/rowid_fetcher.cpp +++ b/be/src/exec/rowid_fetcher.cpp @@ -159,7 +159,8 @@ Status RowIDFetcher::_merge_rpc_results(const PMultiGetRequest& request, return Status::OK(); } // Merge partial blocks - vectorized::Block partial_block(resp.block()); + vectorized::Block partial_block; + RETURN_IF_ERROR(partial_block.deserialize(resp.block())); if (partial_block.is_empty_column()) { return Status::OK(); } diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp index dce7cc5c0e66ed8..9733558284a8fda 100644 --- a/be/src/exec/schema_scanner.cpp +++ b/be/src/exec/schema_scanner.cpp @@ -31,7 +31,9 @@ #include "exec/schema_scanner/schema_columns_scanner.h" #include "exec/schema_scanner/schema_dummy_scanner.h" #include "exec/schema_scanner/schema_files_scanner.h" +#include "exec/schema_scanner/schema_metadata_name_ids_scanner.h" #include "exec/schema_scanner/schema_partitions_scanner.h" +#include "exec/schema_scanner/schema_profiling_scanner.h" #include "exec/schema_scanner/schema_rowsets_scanner.h" #include "exec/schema_scanner/schema_schema_privileges_scanner.h" #include "exec/schema_scanner/schema_schemata_scanner.h" @@ -144,6 +146,10 @@ std::unique_ptr SchemaScanner::create(TSchemaTableType::type type return SchemaPartitionsScanner::create_unique(); case TSchemaTableType::SCH_ROWSETS: return SchemaRowsetsScanner::create_unique(); + case TSchemaTableType::SCH_METADATA_NAME_IDS: + return SchemaMetadataNameIdsScanner::create_unique(); + case TSchemaTableType::SCH_PROFILING: + return SchemaProfilingScanner::create_unique(); default: return SchemaDummyScanner::create_unique(); break; diff --git a/be/src/exec/schema_scanner/schema_helper.cpp b/be/src/exec/schema_scanner/schema_helper.cpp index 9dbc40886e1bc21..3184aed4d2e4f5a 100644 --- a/be/src/exec/schema_scanner/schema_helper.cpp +++ b/be/src/exec/schema_scanner/schema_helper.cpp @@ -60,6 +60,14 @@ Status SchemaHelper::list_table_status(const std::string& ip, const int32_t port client->listTableStatus(*result, request); }); } +Status SchemaHelper::list_table_metadata_name_ids(const std::string& ip, const int32_t port, + const doris::TGetTablesParams& request, + TListTableMetadataNameIdsResult* result) { + return ThriftRpcHelper::rpc( + ip, port, [&request, &result](FrontendServiceConnection& client) { + client->listTableMetadataNameIds(*result, request); + }); +} Status SchemaHelper::describe_table(const std::string& ip, const int32_t port, const TDescribeTableParams& request, diff --git a/be/src/exec/schema_scanner/schema_helper.h b/be/src/exec/schema_scanner/schema_helper.h index 72b7a9acf056c1c..900f963f7893ccd 100644 --- a/be/src/exec/schema_scanner/schema_helper.h +++ b/be/src/exec/schema_scanner/schema_helper.h @@ -34,6 +34,7 @@ class TGetTablesParams; class TGetTablesResult; class TListPrivilegesResult; class TListTableStatusResult; +class TListTableMetadataNameIdsResult; class TShowVariableRequest; class TShowVariableResult; @@ -50,6 +51,9 @@ class SchemaHelper { static Status list_table_status(const std::string& ip, const int32_t port, const TGetTablesParams& table_params, TListTableStatusResult* table_result); + static Status list_table_metadata_name_ids(const std::string& ip, const int32_t port, + const doris::TGetTablesParams& request, + TListTableMetadataNameIdsResult* result); static Status describe_table(const std::string& ip, const int32_t port, const TDescribeTableParams& desc_params, diff --git a/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.cpp b/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.cpp new file mode 100644 index 000000000000000..f99d05dc276bc36 --- /dev/null +++ b/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.cpp @@ -0,0 +1,244 @@ +// 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 "exec/schema_scanner/schema_metadata_name_ids_scanner.h" + +#include +#include +#include + +#include + +#include "common/status.h" +#include "exec/schema_scanner/schema_helper.h" +#include "runtime/decimalv2_value.h" +#include "runtime/define_primitive_type.h" +#include "util/runtime_profile.h" +#include "util/timezone_utils.h" +#include "vec/common/string_ref.h" +#include "vec/runtime/vdatetime_value.h" + +namespace doris { +class RuntimeState; + +namespace vectorized { +class Block; +} // namespace vectorized + +std::vector SchemaMetadataNameIdsScanner::_s_tbls_columns = { + // name, type, size, is_null + {"CATALOG_ID", TYPE_BIGINT, sizeof(int64_t), true}, + {"CATALOG_NAME", TYPE_VARCHAR, sizeof(StringRef), true}, + {"DATABASE_ID", TYPE_BIGINT, sizeof(int64_t), true}, + {"DATABASE_NAME", TYPE_VARCHAR, sizeof(StringRef), true}, + {"TABLE_ID", TYPE_BIGINT, sizeof(int64_t), true}, + {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), true}, +}; + +SchemaMetadataNameIdsScanner::SchemaMetadataNameIdsScanner() + : SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_METADATA_NAME_IDS), _db_index(0) {} + +SchemaMetadataNameIdsScanner::~SchemaMetadataNameIdsScanner() {} + +Status SchemaMetadataNameIdsScanner::start(RuntimeState* state) { + if (!_is_init) { + return Status::InternalError("used before initialized."); + } + SCOPED_TIMER(_get_db_timer); + TGetDbsParams db_params; + if (nullptr != _param->db) { + db_params.__set_pattern(*(_param->db)); + } + if (nullptr != _param->catalog) { + db_params.__set_catalog(*(_param->catalog)); + } + if (nullptr != _param->current_user_ident) { + db_params.__set_current_user_ident(*(_param->current_user_ident)); + } else { + if (nullptr != _param->user) { + db_params.__set_user(*(_param->user)); + } + if (nullptr != _param->user_ip) { + db_params.__set_user_ip(*(_param->user_ip)); + } + } + db_params.__set_get_null_catalog(true); + if (nullptr != _param->ip && 0 != _param->port) { + RETURN_IF_ERROR( + SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result)); + } else { + return Status::InternalError("IP or port doesn't exists"); + } + return Status::OK(); +} + +Status SchemaMetadataNameIdsScanner::_get_new_table() { + SCOPED_TIMER(_get_table_timer); + if (_db_result.db_ids[_db_index] == -1 && + _db_result.dbs[_db_index] == "NULL") { //catalog is empty. + _db_index++; + _table_result.tables.clear(); + _table_result.tables.push_back(TTableMetadataNameIds()); + + return Status::OK(); + } + TGetTablesParams table_params; + table_params.__set_db(_db_result.dbs[_db_index]); + if (_db_result.__isset.catalogs) { + table_params.__set_catalog(_db_result.catalogs[_db_index]); + } + _db_index++; + if (nullptr != _param->wild) { + table_params.__set_pattern(*(_param->wild)); + } + if (nullptr != _param->current_user_ident) { + table_params.__set_current_user_ident(*(_param->current_user_ident)); + } else { + if (nullptr != _param->user) { + table_params.__set_user(*(_param->user)); + } + if (nullptr != _param->user_ip) { + table_params.__set_user_ip(*(_param->user_ip)); + } + } + + if (nullptr != _param->ip && 0 != _param->port) { + RETURN_IF_ERROR(SchemaHelper::list_table_metadata_name_ids(*(_param->ip), _param->port, + table_params, &_table_result)); + } else { + return Status::InternalError("IP or port doesn't exists"); + } + return Status::OK(); +} + +Status SchemaMetadataNameIdsScanner::_fill_block_impl(vectorized::Block* block) { + SCOPED_TIMER(_fill_block_timer); + auto table_num = _table_result.tables.size(); + if (table_num == 0) { //database is null + table_num = 1; + _table_result.tables.push_back(TTableMetadataNameIds()); + } + std::vector null_datas(table_num, nullptr); + std::vector datas(table_num); + + // catalog_id + { + int64_t srcs[table_num]; + if (_db_result.__isset.catalog_ids) { + int64_t id = _db_result.catalog_ids[_db_index - 1]; + for (int i = 0; i < table_num; ++i) { + srcs[i] = id; + datas[i] = srcs + i; + } + fill_dest_column_for_range(block, 0, datas); + } else { + fill_dest_column_for_range(block, 0, null_datas); + } + } + + // catalog_name + { + if (_db_result.__isset.catalogs) { + std::string catalog_name = _db_result.catalogs[_db_index - 1]; + StringRef str_slot = StringRef(catalog_name.c_str(), catalog_name.size()); + for (int i = 0; i < table_num; ++i) { + datas[i] = &str_slot; + } + fill_dest_column_for_range(block, 1, datas); + } else { + fill_dest_column_for_range(block, 1, null_datas); + } + } + + // database_id + { + int64_t srcs[table_num]; + if (_db_result.__isset.db_ids) { + int64_t id = _db_result.db_ids[_db_index - 1]; + for (int i = 0; i < table_num; ++i) { + srcs[i] = id; + datas[i] = srcs + i; + } + fill_dest_column_for_range(block, 2, datas); + } else { + fill_dest_column_for_range(block, 2, null_datas); + } + } + + // database_name + { + if (_db_result.__isset.dbs) { + std::string db_name = SchemaHelper::extract_db_name(_db_result.dbs[_db_index - 1]); + StringRef str_slot = StringRef(db_name.c_str(), db_name.size()); + for (int i = 0; i < table_num; ++i) { + datas[i] = &str_slot; + } + fill_dest_column_for_range(block, 3, datas); + } else { + fill_dest_column_for_range(block, 3, null_datas); + } + } + // table_id + { + int64_t srcs[table_num]; + for (int i = 0; i < table_num; ++i) { + if (_table_result.tables[i].__isset.id) { + srcs[i] = _table_result.tables[i].id; + datas[i] = &srcs; + } else { + datas[i] = nullptr; + } + } + fill_dest_column_for_range(block, 4, datas); + } + + //table_name + { + StringRef strs[table_num]; + for (int i = 0; i < table_num; ++i) { + if (_table_result.tables[i].__isset.name) { + const std::string* src = &_table_result.tables[i].name; + strs[i] = StringRef(src->c_str(), src->size()); + datas[i] = strs + i; + } else { + datas[i] = nullptr; + } + } + fill_dest_column_for_range(block, 5, datas); + } + + return Status::OK(); +} + +Status SchemaMetadataNameIdsScanner::get_next_block(vectorized::Block* block, bool* eos) { + if (!_is_init) { + return Status::InternalError("Used before initialized."); + } + if (nullptr == block || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); + } + if (_db_index < _db_result.dbs.size()) { + RETURN_IF_ERROR(_get_new_table()); + } else { + *eos = true; + return Status::OK(); + } + *eos = false; + return _fill_block_impl(block); +} + +} // namespace doris diff --git a/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.h b/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.h new file mode 100644 index 000000000000000..9981d441d856aa1 --- /dev/null +++ b/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.h @@ -0,0 +1,54 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include + +#include "common/status.h" +#include "exec/schema_scanner.h" + +namespace doris { +class RuntimeState; + +namespace vectorized { +class Block; +} // namespace vectorized + +class SchemaMetadataNameIdsScanner : public SchemaScanner { + ENABLE_FACTORY_CREATOR(SchemaMetadataNameIdsScanner); + +public: + SchemaMetadataNameIdsScanner(); + ~SchemaMetadataNameIdsScanner() override; + + Status start(RuntimeState* state) override; + Status get_next_block(vectorized::Block* block, bool* eos) override; + +private: + Status _get_new_table(); + Status _fill_block_impl(vectorized::Block* block); + + int _db_index; + TGetDbsResult _db_result; + TListTableMetadataNameIdsResult _table_result; + static std::vector _s_tbls_columns; +}; + +} // namespace doris diff --git a/be/src/exec/schema_scanner/schema_profiling_scanner.cpp b/be/src/exec/schema_scanner/schema_profiling_scanner.cpp new file mode 100644 index 000000000000000..d2bd8b256fced13 --- /dev/null +++ b/be/src/exec/schema_scanner/schema_profiling_scanner.cpp @@ -0,0 +1,102 @@ +// 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 "exec/schema_scanner/schema_profiling_scanner.h" + +#include +#include +#include + +#include "exec/schema_scanner/schema_helper.h" +#include "runtime/define_primitive_type.h" +#include "util/runtime_profile.h" +#include "vec/common/string_ref.h" + +namespace doris { +class RuntimeState; +namespace vectorized { +class Block; +} // namespace vectorized + +std::vector SchemaProfilingScanner::_s_tbls_columns = { + // name, type, size, is_null + {"QUERY_ID", TYPE_INT, sizeof(int), false}, + {"SEQ", TYPE_INT, sizeof(int), false}, + {"STATE", TYPE_VARCHAR, sizeof(StringRef), false}, + {"DURATION", TYPE_DOUBLE, sizeof(double), false}, + {"CPU_USER", TYPE_DOUBLE, sizeof(double), true}, + {"CPU_SYSTEM", TYPE_DOUBLE, sizeof(double), true}, + {"CONTEXT_VOLUNTARY", TYPE_INT, sizeof(int), true}, + {"CONTEXT_INVOLUNTARY", TYPE_INT, sizeof(int), true}, + {"BLOCK_OPS_IN", TYPE_INT, sizeof(int), true}, + {"BLOCK_OPS_OUT", TYPE_INT, sizeof(int), true}, + {"MESSAGES_SENT", TYPE_INT, sizeof(int), true}, + {"MESSAGES_RECEIVED", TYPE_INT, sizeof(int), true}, + {"PAGE_FAULTS_MAJOR", TYPE_INT, sizeof(int), true}, + {"PAGE_FAULTS_MINOR", TYPE_INT, sizeof(int), true}, + {"SWAPS", TYPE_INT, sizeof(int), true}, + {"SOURCE_FUNCTION", TYPE_VARCHAR, sizeof(StringRef), false}, + {"SOURCE_FILE", TYPE_VARCHAR, sizeof(StringRef), false}, + {"SOURCE_LINE", TYPE_INT, sizeof(int), true}, +}; + +SchemaProfilingScanner::SchemaProfilingScanner() + : SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_PROFILING) {} + +SchemaProfilingScanner::~SchemaProfilingScanner() {} + +Status SchemaProfilingScanner::start(RuntimeState* state) { + if (!_is_init) { + return Status::InternalError("used before initialized."); + } + SCOPED_TIMER(_get_db_timer); + TGetDbsParams db_params; + if (nullptr != _param->db) { + db_params.__set_pattern(*(_param->db)); + } + if (nullptr != _param->catalog) { + db_params.__set_catalog(*(_param->catalog)); + } + if (nullptr != _param->current_user_ident) { + db_params.__set_current_user_ident(*(_param->current_user_ident)); + } else { + if (nullptr != _param->user) { + db_params.__set_user(*(_param->user)); + } + if (nullptr != _param->user_ip) { + db_params.__set_user_ip(*(_param->user_ip)); + } + } + + if (nullptr == _param->ip || 0 == _param->port) { + return Status::InternalError("IP or port doesn't exists"); + } + return Status::OK(); +} + +Status SchemaProfilingScanner::get_next_block(vectorized::Block* block, bool* eos) { + if (!_is_init) { + return Status::InternalError("Used before initialized."); + } + if (nullptr == block || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); + } + *eos = true; + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/exec/schema_scanner/schema_profiling_scanner.h b/be/src/exec/schema_scanner/schema_profiling_scanner.h new file mode 100644 index 000000000000000..5399cb14eb43f5d --- /dev/null +++ b/be/src/exec/schema_scanner/schema_profiling_scanner.h @@ -0,0 +1,46 @@ +// 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 "common/status.h" +#include "exec/schema_scanner.h" + +namespace doris { +class RuntimeState; +namespace vectorized { +class Block; +} // namespace vectorized + +class SchemaProfilingScanner : public SchemaScanner { + ENABLE_FACTORY_CREATOR(SchemaProfilingScanner); + +public: + SchemaProfilingScanner(); + ~SchemaProfilingScanner() override; + + Status start(RuntimeState* state) override; + Status get_next_block(vectorized::Block* block, bool* eos) override; + + static std::vector _s_tbls_columns; +}; + +} // namespace doris diff --git a/be/src/exec/table_connector.cpp b/be/src/exec/table_connector.cpp index 94c7d889be51b9e..7aa90eda08678a5 100644 --- a/be/src/exec/table_connector.cpp +++ b/be/src/exec/table_connector.cpp @@ -205,7 +205,7 @@ Status TableConnector::convert_column_data(const vectorized::ColumnPtr& column_p // for oracle/pg database string must be ' if (table_type == TOdbcTableType::ORACLE || table_type == TOdbcTableType::POSTGRESQL || table_type == TOdbcTableType::SAP_HANA || table_type == TOdbcTableType::MYSQL || - table_type == TOdbcTableType::CLICKHOUSE) { + table_type == TOdbcTableType::CLICKHOUSE || table_type == TOdbcTableType::SQLSERVER) { fmt::format_to(_insert_stmt_buffer, "'{}'", fmt::basic_string_view(item, size)); } else { fmt::format_to(_insert_stmt_buffer, "\"{}\"", fmt::basic_string_view(item, size)); diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp index 6c24087c01e1dd7..764dddf4e9b9fd2 100644 --- a/be/src/exec/tablet_info.cpp +++ b/be/src/exec/tablet_info.cpp @@ -36,6 +36,7 @@ #include "runtime/types.h" #include "util/hash_util.hpp" #include "util/string_parser.hpp" +#include "util/string_util.h" #include "vec/common/string_ref.h" #include "vec/exprs/vexpr.h" #include "vec/runtime/vdatetime_value.h" @@ -126,7 +127,7 @@ Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) { for (auto& t_slot_desc : tschema.slot_descs) { auto slot_desc = _obj_pool.add(new SlotDescriptor(t_slot_desc)); _tuple_desc->add_slot(slot_desc); - slots_map.emplace(slot_desc->col_name(), slot_desc); + slots_map.emplace(to_lower(slot_desc->col_name()), slot_desc); } for (auto& t_index : tschema.indexes) { @@ -137,7 +138,7 @@ Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) { if (_is_partial_update && _partial_update_input_columns.count(col) == 0) { continue; } - auto it = slots_map.find(col); + auto it = slots_map.find(to_lower(col)); if (it == std::end(slots_map)) { return Status::InternalError("unknown index column, column={}", col); } @@ -154,7 +155,7 @@ Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) { for (auto& tindex_desc : t_index.indexes_desc) { std::vector column_unique_ids(tindex_desc.columns.size()); for (size_t i = 0; i < tindex_desc.columns.size(); i++) { - auto it = slots_map.find(tindex_desc.columns[i]); + auto it = slots_map.find(to_lower(tindex_desc.columns[i])); if (it != std::end(slots_map)) { column_unique_ids[i] = it->second->col_unique_id(); } diff --git a/be/src/exprs/bitmapfilter_predicate.h b/be/src/exprs/bitmapfilter_predicate.h index 561bc7a001cc4ec..743a55c4b6e394d 100644 --- a/be/src/exprs/bitmapfilter_predicate.h +++ b/be/src/exprs/bitmapfilter_predicate.h @@ -19,6 +19,7 @@ #include +#include "exprs/runtime_filter.h" #include "gutil/integral_types.h" #include "runtime/define_primitive_type.h" #include "runtime/primitive_type.h" @@ -27,7 +28,7 @@ namespace doris { // only used in Runtime Filter -class BitmapFilterFuncBase { +class BitmapFilterFuncBase : public FilterFuncBase { public: virtual void insert(const void* data) = 0; virtual void insert_many(const std::vector bitmaps) = 0; @@ -43,13 +44,9 @@ class BitmapFilterFuncBase { void set_not_in(bool not_in) { _not_in = not_in; } virtual ~BitmapFilterFuncBase() = default; - void set_filter_id(int filter_id) { _filter_id = filter_id; } - int get_filter_id() const { return _filter_id; } - protected: // true -> not in bitmap, false -> in bitmap bool _not_in {false}; - int _filter_id = -1; }; template diff --git a/be/src/exprs/bloom_filter_func.h b/be/src/exprs/bloom_filter_func.h index 2d2b7664ace5ec2..0a86a3599025d46 100644 --- a/be/src/exprs/bloom_filter_func.h +++ b/be/src/exprs/bloom_filter_func.h @@ -94,7 +94,7 @@ class BloomFilterAdaptor { }; // Only Used In RuntimeFilter -class BloomFilterFuncBase { +class BloomFilterFuncBase : public FilterFuncBase { public: BloomFilterFuncBase() : _inited(false) {} @@ -164,7 +164,6 @@ class BloomFilterFuncBase { << other_func->_bloom_filter_alloced; return Status::InvalidArgument("bloom filter size invalid"); } - set_filter_id(other_func->_filter_id); return _bloom_filter->merge(other_func->_bloom_filter.get()); } { @@ -176,7 +175,6 @@ class BloomFilterFuncBase { _bloom_filter = bloomfilter_func->_bloom_filter; _bloom_filter_alloced = other_func->_bloom_filter_alloced; _inited = true; - set_filter_id(other_func->_filter_id); return Status::OK(); } else { DCHECK(bloomfilter_func != nullptr); @@ -187,7 +185,6 @@ class BloomFilterFuncBase { << other_func->_bloom_filter_alloced; return Status::InvalidArgument("bloom filter size invalid"); } - set_filter_id(other_func->_filter_id); return _bloom_filter->merge(other_func->_bloom_filter.get()); } } @@ -215,13 +212,8 @@ class BloomFilterFuncBase { _bloom_filter_alloced = other_func->_bloom_filter_alloced; _bloom_filter = other_func->_bloom_filter; _inited = other_func->_inited; - set_filter_id(other_func->_filter_id); } - void set_filter_id(int filter_id) { _filter_id = filter_id; } - - int get_filter_id() const { return _filter_id; } - virtual void insert(const void* data) = 0; // This function is only to be used if the be_exec_version may be less than 2. If updated, please delete it. @@ -255,7 +247,6 @@ class BloomFilterFuncBase { std::mutex _lock; int64_t _bloom_filter_length; bool _build_bf_exactly = false; - int _filter_id = -1; }; template @@ -417,6 +408,7 @@ struct FixedStringFindOp : public StringFindOp { const auto* value = reinterpret_cast(input_data); int64_t size = value->size; const char* data = value->data; + // CHAR type may pad the tail with \0, need to trim while (size > 0 && data[size - 1] == '\0') { size--; } diff --git a/be/src/exprs/create_predicate_function.h b/be/src/exprs/create_predicate_function.h index aefe8e3f0345c57..7d89141c443b37c 100644 --- a/be/src/exprs/create_predicate_function.h +++ b/be/src/exprs/create_predicate_function.h @@ -19,10 +19,12 @@ #include "exprs/hybrid_set.h" #include "exprs/minmax_predicate.h" +#include "function_filter.h" #include "olap/bitmap_filter_predicate.h" #include "olap/bloom_filter_predicate.h" #include "olap/column_predicate.h" #include "olap/in_list_predicate.h" +#include "olap/like_column_predicate.h" #include "runtime/define_primitive_type.h" namespace doris { @@ -256,6 +258,24 @@ ColumnPredicate* create_olap_column_predicate(uint32_t column_id, column->length()); } +template +ColumnPredicate* create_olap_column_predicate(uint32_t column_id, + const std::shared_ptr& filter, int, + const TabletColumn* column = nullptr) { + // currently only support like predicate + if constexpr (PT == TYPE_CHAR || PT == TYPE_VARCHAR || PT == TYPE_STRING) { + if constexpr (PT == TYPE_CHAR) { + return new LikeColumnPredicate(filter->_opposite, column_id, filter->_fn_ctx, + filter->_string_param); + } else { + return new LikeColumnPredicate(filter->_opposite, column_id, + filter->_fn_ctx, filter->_string_param); + } + } else { + return nullptr; + } +} + template ColumnPredicate* create_column_predicate(uint32_t column_id, const std::shared_ptr& filter, FieldType type, int be_exec_version, diff --git a/be/src/exprs/hybrid_set.h b/be/src/exprs/hybrid_set.h index 8084d76ce2711e0..6a90bdd47cdf35c 100644 --- a/be/src/exprs/hybrid_set.h +++ b/be/src/exprs/hybrid_set.h @@ -18,6 +18,7 @@ #pragma once #include "common/object_pool.h" +#include "exprs/runtime_filter.h" #include "runtime/decimalv2_value.h" #include "runtime/define_primitive_type.h" #include "runtime/primitive_type.h" @@ -174,7 +175,7 @@ class DynamicContainer { }; // TODO Maybe change void* parameter to template parameter better. -class HybridSetBase { +class HybridSetBase : public FilterFuncBase { public: HybridSetBase() = default; virtual ~HybridSetBase() = default; @@ -225,9 +226,6 @@ class HybridSetBase { LOG(FATAL) << "HybridSetBase not support find_batch_nullable_negative"; } - void set_filter_id(int filter_id) { _filter_id = filter_id; } - int get_filter_id() const { return _filter_id; } - int _filter_id = -1; class IteratorBase { public: IteratorBase() = default; diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index e1abc0f55dbf11c..3df28bfd37b72b7 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1490,7 +1490,6 @@ void IRuntimeFilter::init_profile(RuntimeProfile* parent_profile) { void IRuntimeFilter::update_runtime_filter_type_to_profile() { if (_profile != nullptr) { _profile->add_info_string("RealRuntimeFilterType", to_string(_wrapper->get_real_type())); - _wrapper->set_filter_id(_filter_id); } } @@ -1845,7 +1844,7 @@ Status RuntimePredicateWrapper::get_push_exprs(std::listroot()->type().type == _column_return_type || (is_string_type(probe_ctx->root()->type().type) && is_string_type(_column_return_type)) || diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index 6dc074447321eff..3bd3efd6cc8cf95 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -97,7 +97,19 @@ struct RuntimeFilterParams { bool bitmap_filter_not_in; bool build_bf_exactly; }; +struct FilterFuncBase { +public: + void set_filter_id(int filter_id) { + if (_filter_id == -1) { + _filter_id = filter_id; + } + } + [[nodiscard]] int get_filter_id() const { return _filter_id; } + +private: + int _filter_id = -1; +}; struct UpdateRuntimeFilterParams { UpdateRuntimeFilterParams(const PPublishFilterRequest* req, butil::IOBufAsZeroCopyInputStream* data_stream, ObjectPool* obj_pool) diff --git a/be/src/exprs/runtime_filter_slots.h b/be/src/exprs/runtime_filter_slots.h index c6a99770ee571e8..fe71f683a0a20cc 100644 --- a/be/src/exprs/runtime_filter_slots.h +++ b/be/src/exprs/runtime_filter_slots.h @@ -33,21 +33,11 @@ namespace doris { class VRuntimeFilterSlots { public: VRuntimeFilterSlots( - const std::vector>& prob_expr_ctxs, const std::vector>& build_expr_ctxs, const std::vector& runtime_filter_descs) - : _probe_expr_context(prob_expr_ctxs), - _build_expr_context(build_expr_ctxs), - _runtime_filter_descs(runtime_filter_descs) {} + : _build_expr_context(build_expr_ctxs), _runtime_filter_descs(runtime_filter_descs) {} Status init(RuntimeState* state, int64_t hash_table_size, size_t build_bf_cardinality) { - if (_probe_expr_context.size() != _build_expr_context.size()) { - return Status::InternalError( - "_probe_expr_context.size() != _build_expr_context.size(), " - "_probe_expr_context.size()={}, _build_expr_context.size()={}", - _probe_expr_context.size(), _build_expr_context.size()); - } - // runtime filter effect strategy // 1. we will ignore IN filter when hash_table_size is too big // 2. we will ignore BLOOM filter and MinMax filter when hash_table_size @@ -101,11 +91,11 @@ class VRuntimeFilterSlots { RETURN_IF_ERROR(state->runtime_filter_mgr()->get_producer_filter(filter_desc.filter_id, &runtime_filter)); if (runtime_filter->expr_order() < 0 || - runtime_filter->expr_order() >= _probe_expr_context.size()) { + runtime_filter->expr_order() >= _build_expr_context.size()) { return Status::InternalError( "runtime_filter meet invalid expr_order, expr_order={}, " - "probe_expr_context.size={}", - runtime_filter->expr_order(), _probe_expr_context.size()); + "_build_expr_context.size={}", + runtime_filter->expr_order(), _build_expr_context.size()); } // do not create 'in filter' when hash_table size over limit @@ -262,7 +252,6 @@ class VRuntimeFilterSlots { bool empty() { return !_runtime_filters.size(); } private: - const std::vector>& _probe_expr_context; const std::vector>& _build_expr_context; const std::vector& _runtime_filter_descs; // prob_contition index -> [IRuntimeFilter] diff --git a/be/src/http/action/stream_load_with_sql.cpp b/be/src/http/action/http_stream.cpp similarity index 52% rename from be/src/http/action/stream_load_with_sql.cpp rename to be/src/http/action/http_stream.cpp index cf07f5a39a5bbd2..f8068389b2a76ea 100644 --- a/be/src/http/action/stream_load_with_sql.cpp +++ b/be/src/http/action/http_stream.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "http/action/stream_load_with_sql.h" +#include "http/action/http_stream.h" #include #include @@ -65,30 +65,26 @@ #include "util/time.h" #include "util/uid_util.h" -// TODO The functions in this file need to be improved namespace doris { using namespace ErrorCode; -DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(streaming_load_with_sql_requests_total, MetricUnit::REQUESTS); -DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(streaming_load_with_sql_duration_ms, MetricUnit::MILLISECONDS); -DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(streaming_load_with_sql_current_processing, - MetricUnit::REQUESTS); - -StreamLoadWithSqlAction::StreamLoadWithSqlAction(ExecEnv* exec_env) : _exec_env(exec_env) { - _stream_load_with_sql_entity = - DorisMetrics::instance()->metric_registry()->register_entity("stream_load_with_sql"); - INT_COUNTER_METRIC_REGISTER(_stream_load_with_sql_entity, - streaming_load_with_sql_requests_total); - INT_COUNTER_METRIC_REGISTER(_stream_load_with_sql_entity, streaming_load_with_sql_duration_ms); - INT_GAUGE_METRIC_REGISTER(_stream_load_with_sql_entity, - streaming_load_with_sql_current_processing); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(http_stream_requests_total, MetricUnit::REQUESTS); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(http_stream_duration_ms, MetricUnit::MILLISECONDS); +DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(http_stream_current_processing, MetricUnit::REQUESTS); + +HttpStreamAction::HttpStreamAction(ExecEnv* exec_env) : _exec_env(exec_env) { + _http_stream_entity = + DorisMetrics::instance()->metric_registry()->register_entity("http_stream"); + INT_COUNTER_METRIC_REGISTER(_http_stream_entity, http_stream_requests_total); + INT_COUNTER_METRIC_REGISTER(_http_stream_entity, http_stream_duration_ms); + INT_GAUGE_METRIC_REGISTER(_http_stream_entity, http_stream_current_processing); } -StreamLoadWithSqlAction::~StreamLoadWithSqlAction() { - DorisMetrics::instance()->metric_registry()->deregister_entity(_stream_load_with_sql_entity); +HttpStreamAction::~HttpStreamAction() { + DorisMetrics::instance()->metric_registry()->deregister_entity(_http_stream_entity); } -void StreamLoadWithSqlAction::handle(HttpRequest* req) { +void HttpStreamAction::handle(HttpRequest* req) { std::shared_ptr ctx = std::static_pointer_cast(req->handler_ctx()); if (ctx == nullptr) { @@ -118,27 +114,6 @@ void StreamLoadWithSqlAction::handle(HttpRequest* req) { HttpChannel::send_reply(req, str); return; } - - // query stream load status - // put request - TStreamLoadWithLoadStatusRequest request; - TStreamLoadWithLoadStatusResult result; - request.__set_loadId(ctx->id.to_thrift()); - TNetworkAddress master_addr = _exec_env->master_info()->network_address; - ThriftRpcHelper::rpc( - master_addr.hostname, master_addr.port, - [&request, &result](FrontendServiceConnection& client) { - client->streamLoadWithLoadStatus(result, request); - }); - Status stream_load_status(Status::create(result.status)); - if (stream_load_status.ok()) { - ctx->txn_id = result.txn_id; - ctx->number_total_rows = result.total_rows; - ctx->number_loaded_rows = result.loaded_rows; - ctx->number_filtered_rows = result.filtered_rows; - ctx->number_unselected_rows = result.unselected_rows; - } - auto str = std::string(ctx->to_json()); // add new line at end str = str + '\n'; @@ -148,34 +123,30 @@ void StreamLoadWithSqlAction::handle(HttpRequest* req) { _save_stream_load_record(ctx, str); } // update statistics - streaming_load_with_sql_requests_total->increment(1); - streaming_load_with_sql_duration_ms->increment(ctx->load_cost_millis); - streaming_load_with_sql_current_processing->increment(-1); + http_stream_requests_total->increment(1); + http_stream_duration_ms->increment(ctx->load_cost_millis); + http_stream_current_processing->increment(-1); } -Status StreamLoadWithSqlAction::_handle(HttpRequest* req, std::shared_ptr ctx) { +Status HttpStreamAction::_handle(HttpRequest* http_req, std::shared_ptr ctx) { if (ctx->body_bytes > 0 && ctx->receive_bytes != ctx->body_bytes) { LOG(WARNING) << "recevie body don't equal with body bytes, body_bytes=" << ctx->body_bytes << ", receive_bytes=" << ctx->receive_bytes << ", id=" << ctx->id; return Status::InternalError("receive body don't equal with body bytes"); } - if (!ctx->use_streaming) { - // if we use non-streaming, we need to close file first, - // then execute_plan_fragment here - // this will close file - ctx->body_sink.reset(); - // TODO This function may not be placed here - _process_put(req, ctx); - } else { - RETURN_IF_ERROR(ctx->body_sink->finish()); - } - // TODO support parquet and orc + RETURN_IF_ERROR(ctx->body_sink->finish()); + + // wait stream load finish RETURN_IF_ERROR(ctx->future.get()); - return ctx->status; + + 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; + return Status::OK(); } -int StreamLoadWithSqlAction::on_header(HttpRequest* req) { - streaming_load_with_sql_current_processing->increment(1); +int HttpStreamAction::on_header(HttpRequest* req) { + http_stream_current_processing->increment(1); std::shared_ptr ctx = std::make_shared(_exec_env); req->set_handler_ctx(ctx); @@ -188,8 +159,6 @@ int StreamLoadWithSqlAction::on_header(HttpRequest* req) { ctx->label = generate_uuid_string(); } - ctx->two_phase_commit = req->header(HTTP_TWO_PHASE_COMMIT) == "true" ? true : false; - LOG(INFO) << "new income streaming load request." << ctx->brief() << " sql : " << req->header(HTTP_SQL); @@ -203,7 +172,7 @@ int StreamLoadWithSqlAction::on_header(HttpRequest* req) { // add new line at end str = str + '\n'; HttpChannel::send_reply(req, str); - streaming_load_with_sql_current_processing->increment(-1); + http_stream_current_processing->increment(-1); if (config::enable_stream_load_record) { str = ctx->prepare_stream_load_record(str); _save_stream_load_record(ctx, str); @@ -213,85 +182,42 @@ int StreamLoadWithSqlAction::on_header(HttpRequest* req) { return 0; } -// TODO The parameters of this function may need to be refactored because the parameters in HttpRequest are not sufficient. -Status StreamLoadWithSqlAction::_on_header(HttpRequest* http_req, - std::shared_ptr ctx) { - // get format of this put - if (!http_req->header(HTTP_COMPRESS_TYPE).empty() && - iequal(http_req->header(HTTP_FORMAT_KEY), "JSON")) { - return Status::InternalError("compress data of JSON format is not supported."); - } - std::string format_str = http_req->header(HTTP_FORMAT_KEY); - if (iequal(format_str, BeConsts::CSV_WITH_NAMES) || - iequal(format_str, BeConsts::CSV_WITH_NAMES_AND_TYPES)) { - ctx->header_type = format_str; - //treat as CSV - format_str = BeConsts::CSV; - } - LoadUtil::parse_format(format_str, http_req->header(HTTP_COMPRESS_TYPE), &ctx->format, - &ctx->compress_type); - if (ctx->format == TFileFormatType::FORMAT_UNKNOWN) { - return Status::InternalError("unknown data format, format={}", - http_req->header(HTTP_FORMAT_KEY)); +Status HttpStreamAction::_on_header(HttpRequest* http_req, std::shared_ptr ctx) { + // auth information + if (!parse_basic_auth(*http_req, &ctx->auth)) { + LOG(WARNING) << "parse basic authorization failed." << ctx->brief(); + return Status::InternalError("no valid Basic authorization"); } + // TODO(zs) : need Need to request an FE to obtain information such as format // check content length ctx->body_bytes = 0; size_t csv_max_body_bytes = config::streaming_load_max_mb * 1024 * 1024; - size_t json_max_body_bytes = config::streaming_load_json_max_mb * 1024 * 1024; - bool read_json_by_line = false; - if (!http_req->header(HTTP_READ_JSON_BY_LINE).empty()) { - if (iequal(http_req->header(HTTP_READ_JSON_BY_LINE), "true")) { - read_json_by_line = true; - } - } if (!http_req->header(HttpHeaders::CONTENT_LENGTH).empty()) { ctx->body_bytes = std::stol(http_req->header(HttpHeaders::CONTENT_LENGTH)); - // json max body size - if ((ctx->format == TFileFormatType::FORMAT_JSON) && - (ctx->body_bytes > json_max_body_bytes) && !read_json_by_line) { - return Status::InternalError( - "The size of this batch exceed the max size [{}] of json type data " - " data [ {} ]. Split the file, or use 'read_json_by_line'", - json_max_body_bytes, ctx->body_bytes); - } // csv max body size - else if (ctx->body_bytes > csv_max_body_bytes) { + if (ctx->body_bytes > csv_max_body_bytes) { LOG(WARNING) << "body exceed max size." << ctx->brief(); return Status::InternalError("body exceed max size: {}, data: {}", csv_max_body_bytes, ctx->body_bytes); } - } else { - evhttp_connection_set_max_body_size( - evhttp_request_get_connection(http_req->get_evhttp_request()), csv_max_body_bytes); } - if (!http_req->header(HTTP_TIMEOUT).empty()) { - try { - ctx->timeout_second = std::stoi(http_req->header(HTTP_TIMEOUT)); - } catch (const std::invalid_argument& e) { - return Status::InvalidArgument("Invalid timeout format, {}", e.what()); - } - } + auto pipe = std::make_shared( + io::kMaxPipeBufferedBytes /* max_buffered_bytes */, 64 * 1024 /* min_chunk_size */, + ctx->body_bytes /* total_length */); + ctx->body_sink = pipe; + ctx->pipe = pipe; - ctx->use_streaming = LoadUtil::is_format_support_streaming(ctx->format); - if (ctx->use_streaming) { - // create stream load pipe for fetch schema - auto pipe = std::make_shared( - io::kMaxPipeBufferedBytes /* max_buffered_bytes */, 64 * 1024 /* min_chunk_size */, - ctx->body_bytes /* total_length */); - ctx->body_sink = pipe; - ctx->pipe = pipe; - } else { - // TODO here need _data_saved_path function and file_sink - } RETURN_IF_ERROR(_exec_env->new_load_stream_mgr()->put(ctx->id, ctx)); - ctx->txn_id = 0; + + // Here, transactions are set from fe's NativeInsertStmt. + // TODO(zs) : How to support two_phase_commit return Status::OK(); } -void StreamLoadWithSqlAction::on_chunk_data(HttpRequest* req) { +void HttpStreamAction::on_chunk_data(HttpRequest* req) { std::shared_ptr ctx = std::static_pointer_cast(req->handler_ctx()); if (ctx == nullptr || !ctx->status.ok()) { @@ -302,12 +228,25 @@ void StreamLoadWithSqlAction::on_chunk_data(HttpRequest* req) { 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); bb->pos = remove_bytes; bb->flip(); auto st = ctx->body_sink->append(bb); + // schema_buffer stores 1M of data for parsing column information + // need to determine whether to cache for the first time + if (ctx->is_read_schema) { + if (ctx->schema_buffer->pos + remove_bytes < config::stream_tvf_buffer_size) { + ctx->schema_buffer->put_bytes(bb->ptr, remove_bytes); + } else { + ctx->need_schema = true; + ctx->is_read_schema = false; + ctx->status = _process_put(req, ctx); + } + } + if (!st.ok()) { LOG(WARNING) << "append body content failed. errmsg=" << st << ", " << ctx->brief(); ctx->status = st; @@ -315,10 +254,16 @@ void StreamLoadWithSqlAction::on_chunk_data(HttpRequest* req) { } ctx->receive_bytes += remove_bytes; } + // after all the data has been read and it has not reached 1M, it will execute here + if (ctx->is_read_schema) { + ctx->need_schema = true; + ctx->is_read_schema = false; + ctx->status = _process_put(req, ctx); + } ctx->read_data_cost_nanos += (MonotonicNanos() - start_read_data_time); } -void StreamLoadWithSqlAction::free_handler_ctx(std::shared_ptr param) { +void HttpStreamAction::free_handler_ctx(std::shared_ptr param) { std::shared_ptr ctx = std::static_pointer_cast(param); if (ctx == nullptr) { return; @@ -331,13 +276,10 @@ void StreamLoadWithSqlAction::free_handler_ctx(std::shared_ptr param) { ctx->exec_env()->new_load_stream_mgr()->remove(ctx->id); } -Status StreamLoadWithSqlAction::_process_put(HttpRequest* http_req, - std::shared_ptr ctx) { - // put request +Status HttpStreamAction::_process_put(HttpRequest* http_req, + std::shared_ptr ctx) { TStreamLoadPutRequest request; set_request_auth(&request, ctx->auth); - request.txnId = ctx->txn_id; - request.__set_version(1); request.__set_load_sql(http_req->header(HTTP_SQL)); request.__set_loadId(ctx->id.to_thrift()); request.__set_label(ctx->label); @@ -346,26 +288,8 @@ Status StreamLoadWithSqlAction::_process_put(HttpRequest* http_req, } else { LOG(WARNING) << "_exec_env->master_info not set backend_id"; } - if (!http_req->header(HTTP_EXEC_MEM_LIMIT).empty()) { - try { - request.__set_execMemLimit(std::stoll(http_req->header(HTTP_EXEC_MEM_LIMIT))); - } catch (const std::invalid_argument& e) { - return Status::InvalidArgument("Invalid mem limit format, {}", e.what()); - } - } - if (ctx->use_streaming) { - request.fileType = TFileType::FILE_STREAM; - } else { - request.__isset.path = true; - request.fileType = TFileType::FILE_LOCAL; - request.__set_file_size(ctx->body_bytes); - } - if (ctx->timeout_second != -1) { - request.__set_timeout(ctx->timeout_second); - } - request.__set_thrift_rpc_timeout_ms(config::thrift_rpc_timeout_ms); - // exec this load + // plan this load TNetworkAddress master_addr = _exec_env->master_info()->network_address; int64_t stream_load_put_start_time = MonotonicNanos(); RETURN_IF_ERROR(ThriftRpcHelper::rpc( @@ -376,32 +300,17 @@ Status StreamLoadWithSqlAction::_process_put(HttpRequest* http_req, ctx->stream_load_put_cost_nanos = MonotonicNanos() - stream_load_put_start_time; Status plan_status(Status::create(ctx->put_result.status)); if (!plan_status.ok()) { - LOG(WARNING) << "exec streaming load failed. errmsg=" << plan_status << ctx->brief(); + LOG(WARNING) << "plan streaming load failed. errmsg=" << plan_status << ctx->brief(); return plan_status; } - // TODO perhaps the `execute_plan_fragment` function needs to be executed here - return Status::OK(); -} - -Status StreamLoadWithSqlAction::_data_saved_path(HttpRequest* req, std::string* file_path) { - std::string prefix; - RETURN_IF_ERROR( - _exec_env->load_path_mgr()->allocate_dir("stream_load_local_file", "", &prefix)); - timeval tv; - gettimeofday(&tv, nullptr); - struct tm tm; - time_t cur_sec = tv.tv_sec; - localtime_r(&cur_sec, &tm); - char buf[64]; - strftime(buf, 64, "%Y%m%d%H%M%S", &tm); - std::stringstream ss; - ss << prefix << buf << "." << tv.tv_usec; - *file_path = ss.str(); - return Status::OK(); + 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; + return _exec_env->stream_load_executor()->execute_plan_fragment(ctx); } -void StreamLoadWithSqlAction::_save_stream_load_record(std::shared_ptr ctx, - const std::string& str) { +void HttpStreamAction::_save_stream_load_record(std::shared_ptr ctx, + const std::string& str) { auto stream_load_recorder = StorageEngine::instance()->get_stream_load_recorder(); if (stream_load_recorder != nullptr) { std::string key = diff --git a/be/src/http/action/stream_load_with_sql.h b/be/src/http/action/http_stream.h similarity index 84% rename from be/src/http/action/stream_load_with_sql.h rename to be/src/http/action/http_stream.h index c3afdd7954bde55..192e68257eaf180 100644 --- a/be/src/http/action/stream_load_with_sql.h +++ b/be/src/http/action/http_stream.h @@ -30,10 +30,10 @@ class ExecEnv; class Status; class StreamLoadContext; -class StreamLoadWithSqlAction : public HttpHandler { +class HttpStreamAction : public HttpHandler { public: - StreamLoadWithSqlAction(ExecEnv* exec_env); - ~StreamLoadWithSqlAction() override; + HttpStreamAction(ExecEnv* exec_env); + ~HttpStreamAction() override; void handle(HttpRequest* req) override; @@ -56,10 +56,10 @@ class StreamLoadWithSqlAction : public HttpHandler { private: ExecEnv* _exec_env; - std::shared_ptr _stream_load_with_sql_entity; - IntCounter* streaming_load_with_sql_requests_total; - IntCounter* streaming_load_with_sql_duration_ms; - IntGauge* streaming_load_with_sql_current_processing; + std::shared_ptr _http_stream_entity; + IntCounter* http_stream_requests_total; + IntCounter* http_stream_duration_ms; + IntGauge* http_stream_current_processing; }; } // namespace doris diff --git a/be/src/http/action/reload_tablet_action.cpp b/be/src/http/action/reload_tablet_action.cpp index de54ad66bb717f4..101f1639a556f84 100644 --- a/be/src/http/action/reload_tablet_action.cpp +++ b/be/src/http/action/reload_tablet_action.cpp @@ -94,7 +94,7 @@ void ReloadTabletAction::reload(const std::string& path, int64_t tablet_id, int3 clone_req.__set_schema_hash(schema_hash); Status res = Status::OK(); - res = _exec_env->storage_engine()->load_header(path, clone_req); + res = StorageEngine::instance()->load_header(path, clone_req); if (!res.ok()) { LOG(WARNING) << "load header failed. status: " << res << ", signature: " << tablet_id; std::string error_msg = std::string("load header failed"); diff --git a/be/src/http/action/reload_tablet_action.h b/be/src/http/action/reload_tablet_action.h index 6c984fbf27ceb19..36ed12680a3b3d1 100644 --- a/be/src/http/action/reload_tablet_action.h +++ b/be/src/http/action/reload_tablet_action.h @@ -39,7 +39,6 @@ class ReloadTabletAction : public HttpHandlerWithAuth { private: void reload(const std::string& path, int64_t tablet_id, int32_t schema_hash, HttpRequest* req); - ExecEnv* _exec_env; }; // end class ReloadTabletAction } // end namespace doris diff --git a/be/src/http/action/restore_tablet_action.cpp b/be/src/http/action/restore_tablet_action.cpp index 1400c75c619b27c..d8c5dbcd2b08bf5 100644 --- a/be/src/http/action/restore_tablet_action.cpp +++ b/be/src/http/action/restore_tablet_action.cpp @@ -120,7 +120,7 @@ Status RestoreTabletAction::_reload_tablet(const std::string& key, const std::st clone_req.__set_tablet_id(tablet_id); clone_req.__set_schema_hash(schema_hash); Status res = Status::OK(); - res = _exec_env->storage_engine()->load_header(shard_path, clone_req, /*restore=*/true); + res = StorageEngine::instance()->load_header(shard_path, clone_req, /*restore=*/true); if (!res.ok()) { LOG(WARNING) << "load header failed. status: " << res << ", signature: " << tablet_id; // remove tablet data path in data path diff --git a/be/src/http/action/restore_tablet_action.h b/be/src/http/action/restore_tablet_action.h index 845061789ffa0a7..9ba6ea6a5fc02f9 100644 --- a/be/src/http/action/restore_tablet_action.h +++ b/be/src/http/action/restore_tablet_action.h @@ -58,7 +58,6 @@ class RestoreTabletAction : public HttpHandlerWithAuth { Status _create_hard_link_recursive(const std::string& src, const std::string& dst); private: - ExecEnv* _exec_env; std::mutex _tablet_restore_lock; // store all current restoring tablet_id + schema_hash // key: tablet_id + schema_hash diff --git a/be/src/http/action/tablet_migration_action.cpp b/be/src/http/action/tablet_migration_action.cpp index 9720b8863d81332..3d5b40ae186624c 100644 --- a/be/src/http/action/tablet_migration_action.cpp +++ b/be/src/http/action/tablet_migration_action.cpp @@ -171,9 +171,9 @@ Status TabletMigrationAction::_check_param(HttpRequest* req, int64_t& tablet_id, tablet_id = std::stoull(req_tablet_id); schema_hash = std::stoul(req_schema_hash); } catch (const std::exception& e) { - LOG(WARNING) << "invalid argument.tablet_id:" << req_tablet_id - << ", schema_hash:" << req_schema_hash; - return Status::InternalError("Convert failed, {}", e.what()); + return Status::InternalError( + "Convert failed:{}, invalid argument.tablet_id: {}, schema_hash: {}", e.what(), + req_tablet_id, req_schema_hash); } dest_disk = req->param("disk"); goal = req->param("goal"); diff --git a/be/src/http/action/tablets_distribution_action.cpp b/be/src/http/action/tablets_distribution_action.cpp index b6c88ae65f496ba..4c1f039462e26cb 100644 --- a/be/src/http/action/tablets_distribution_action.cpp +++ b/be/src/http/action/tablets_distribution_action.cpp @@ -116,9 +116,6 @@ EasyJson TabletsDistributionAction::get_tablets_distribution_group_by_partition( tablet["tablet_id"] = tablets_info_on_disk[partition_iter->first][disk_iter->first][i] .tablet_id; - tablet["schema_hash"] = - tablets_info_on_disk[partition_iter->first][disk_iter->first][i] - .schema_hash; tablet["tablet_size"] = tablets_info_on_disk[partition_iter->first][disk_iter->first][i] .tablet_size; diff --git a/be/src/http/action/tablets_info_action.cpp b/be/src/http/action/tablets_info_action.cpp index c75baee954293a7..519e86f17db6ab3 100644 --- a/be/src/http/action/tablets_info_action.cpp +++ b/be/src/http/action/tablets_info_action.cpp @@ -79,7 +79,6 @@ EasyJson TabletsInfoAction::get_tablets_info(string tablet_num_to_return) { for (TabletInfo tablet_info : tablets_info) { EasyJson tablet = tablets.PushBack(EasyJson::kObject); tablet["tablet_id"] = tablet_info.tablet_id; - tablet["schema_hash"] = tablet_info.schema_hash; } tablets_info_ej["count"] = tablets_info.size(); return tablets_info_ej; diff --git a/be/src/io/file_factory.cpp b/be/src/io/file_factory.cpp index 5838c666ebb7fa6..9c49f0dd8b77d6e 100644 --- a/be/src/io/file_factory.cpp +++ b/be/src/io/file_factory.cpp @@ -150,7 +150,18 @@ Status FileFactory::create_pipe_reader(const TUniqueId& load_id, io::FileReaderS if (!stream_load_ctx) { return Status::InternalError("unknown stream load id: {}", UniqueId(load_id).to_string()); } - *file_reader = stream_load_ctx->pipe; + if (stream_load_ctx->need_schema == true) { + auto pipe = std::make_shared( + io::kMaxPipeBufferedBytes /* max_buffered_bytes */, 64 * 1024 /* min_chunk_size */, + stream_load_ctx->schema_buffer->pos /* total_length */); + stream_load_ctx->schema_buffer->flip(); + pipe->append(stream_load_ctx->schema_buffer); + pipe->finish(); + *file_reader = std::move(pipe); + stream_load_ctx->need_schema = false; + } else { + *file_reader = stream_load_ctx->pipe; + } if (file_reader->get() == nullptr) { return Status::OK(); diff --git a/be/src/io/fs/buffered_reader.cpp b/be/src/io/fs/buffered_reader.cpp index 726f5331c96f799..00f88c75154b269 100644 --- a/be/src/io/fs/buffered_reader.cpp +++ b/be/src/io/fs/buffered_reader.cpp @@ -778,8 +778,12 @@ Status DelegateReader::create_file_reader(RuntimeProfile* profile, io::FileReaderSPtr reader; RETURN_IF_ERROR(FileFactory::create_file_reader(system_properties, file_description, reader_options, file_system, &reader, profile)); - if (reader->size() < IN_MEMORY_FILE_SIZE) { - *file_reader = std::make_shared(reader); + if (reader->size() < config::in_memory_file_size) { + if (typeid_cast(reader.get())) { + *file_reader = std::make_shared(reader); + } else { + *file_reader = std::move(reader); + } } else if (access_mode == AccessMode::SEQUENTIAL) { bool is_thread_safe = false; if (typeid_cast(reader.get())) { diff --git a/be/src/io/fs/buffered_reader.h b/be/src/io/fs/buffered_reader.h index 34e1ff34fede6d8..25a681133037f3c 100644 --- a/be/src/io/fs/buffered_reader.h +++ b/be/src/io/fs/buffered_reader.h @@ -238,7 +238,7 @@ class MergeRangeFileReader : public io::FileReader { /** * Create a file reader suitable for accessing scenarios: - * 1. When file size < 8MB, create InMemoryFileReader file reader + * 1. When file size < config::in_memory_file_size, create InMemoryFileReader file reader * 2. When reading sequential file(csv/json), create PrefetchBufferedReader * 3. When reading random access file(parquet/orc), create normal file reader */ @@ -246,8 +246,6 @@ class DelegateReader { public: enum AccessMode { SEQUENTIAL, RANDOM }; - static constexpr size_t IN_MEMORY_FILE_SIZE = 8 * 1024 * 1024; - static Status create_file_reader( RuntimeProfile* profile, const FileSystemProperties& system_properties, const FileDescription& file_description, const io::FileReaderOptions& reader_options, diff --git a/be/src/io/fs/hdfs_file_reader.cpp b/be/src/io/fs/hdfs_file_reader.cpp index 383958802f25523..6c4f456e3779d10 100644 --- a/be/src/io/fs/hdfs_file_reader.cpp +++ b/be/src/io/fs/hdfs_file_reader.cpp @@ -133,13 +133,21 @@ Status HdfsFileReader::read_at_impl(size_t offset, Slice result, size_t* bytes_r return Status::OK(); } - tSize r = hdfsPread(_handle->fs(), _handle->file(), offset, to, bytes_req); - if (r == -1) { - return Status::InternalError( - "Read hdfs file failed. (BE: {}) namenode:{}, path:{}, err: {}", - BackendOptions::get_localhost(), _name_node, _path.string(), hdfs_error()); + size_t has_read = 0; + while (has_read < bytes_req) { + tSize loop_read = hdfsPread(_handle->fs(), _handle->file(), offset + has_read, + to + has_read, bytes_req - has_read); + if (loop_read < 0) { + return Status::InternalError( + "Read hdfs file failed. (BE: {}) namenode:{}, path:{}, err: {}", + BackendOptions::get_localhost(), _name_node, _path.string(), hdfs_error()); + } + if (loop_read == 0) { + break; + } + has_read += loop_read; } - *bytes_read = bytes_req; + *bytes_read = has_read; return Status::OK(); } diff --git a/be/src/io/fs/stream_sink_file_writer.cpp b/be/src/io/fs/stream_sink_file_writer.cpp index ff2667b9fa3af1b..672693351367dfc 100644 --- a/be/src/io/fs/stream_sink_file_writer.cpp +++ b/be/src/io/fs/stream_sink_file_writer.cpp @@ -21,6 +21,7 @@ #include "olap/olap_common.h" #include "olap/rowset/beta_rowset_writer.h" +#include "vec/sink/load_stream_stub.h" namespace doris { namespace io { @@ -35,38 +36,24 @@ void StreamSinkFileWriter::init(PUniqueId load_id, int64_t partition_id, int64_t _index_id = index_id; _tablet_id = tablet_id; _segment_id = segment_id; - - _header.set_src_id(_sender_id); - *_header.mutable_load_id() = _load_id; - _header.set_partition_id(_partition_id); - _header.set_index_id(_index_id); - _header.set_tablet_id(_tablet_id); - _header.set_segment_id(_segment_id); - _header.set_opcode(doris::PStreamHeader::APPEND_DATA); - _append_header(); } Status StreamSinkFileWriter::appendv(const Slice* data, size_t data_cnt) { size_t bytes_req = 0; for (int i = 0; i < data_cnt; i++) { bytes_req += data[i].get_size(); - _buf.append(data[i].get_data(), data[i].get_size()); } - _pending_bytes += bytes_req; _bytes_appended += bytes_req; VLOG_DEBUG << "writer appendv, load_id: " << UniqueId(_load_id).to_string() << ", index_id: " << _index_id << ", tablet_id: " << _tablet_id - << ", segment_id: " << _segment_id << ", data_length: " << bytes_req - << ", current batched bytes: " << _pending_bytes; + << ", segment_id: " << _segment_id << ", data_length: " << bytes_req; - if (_pending_bytes >= _max_pending_bytes) { - RETURN_IF_ERROR(_stream_sender(_buf)); - _buf.clear(); - _append_header(); - _pending_bytes = 0; + std::span slices {data, data_cnt}; + for (auto& stream : _streams) { + RETURN_IF_ERROR( + stream->append_data(_partition_id, _index_id, _tablet_id, _segment_id, slices)); } - return Status::OK(); } @@ -75,38 +62,11 @@ Status StreamSinkFileWriter::finalize() { << ", index_id: " << _index_id << ", tablet_id: " << _tablet_id << ", segment_id: " << _segment_id; // TODO(zhengyu): update get_inverted_index_file_size into stat - Status status = _stream_sender(_buf); - // send eos - _buf.clear(); - _header.set_segment_eos(true); - _append_header(); - status = _stream_sender(_buf); - return status; -} - -void StreamSinkFileWriter::_append_header() { - size_t header_len = _header.ByteSizeLong(); - _buf.append(reinterpret_cast(&header_len), sizeof(header_len)); - _buf.append(_header.SerializeAsString()); -} - -Status StreamSinkFileWriter::send_with_retry(brpc::StreamId stream, butil::IOBuf buf) { - while (true) { - int ret = brpc::StreamWrite(stream, buf); - if (ret == EAGAIN) { - const timespec time = butil::seconds_from_now(60); - int wait_result = brpc::StreamWait(stream, &time); - if (wait_result == 0) { - continue; - } else { - return Status::InternalError("fail to send data when wait stream"); - } - } else if (ret == EINVAL) { - return Status::InternalError("fail to send data when stream write"); - } else { - return Status::OK(); - } + for (auto& stream : _streams) { + RETURN_IF_ERROR( + stream->append_data(_partition_id, _index_id, _tablet_id, _segment_id, {}, true)); } + return Status::OK(); } Status StreamSinkFileWriter::close() { diff --git a/be/src/io/fs/stream_sink_file_writer.h b/be/src/io/fs/stream_sink_file_writer.h index 074a2f0f5add3e7..6c40543b936850c 100644 --- a/be/src/io/fs/stream_sink_file_writer.h +++ b/be/src/io/fs/stream_sink_file_writer.h @@ -27,18 +27,16 @@ namespace doris { +class LoadStreamStub; + struct RowsetId; struct SegmentStatistics; namespace io { class StreamSinkFileWriter : public FileWriter { public: - StreamSinkFileWriter(int sender_id, std::vector stream_id) - : _sender_id(sender_id), _streams(stream_id) {} - - static void deleter(void* data) { ::free(data); } - - static Status send_with_retry(brpc::StreamId stream, butil::IOBuf buf); + StreamSinkFileWriter(std::vector>& streams) + : _streams(streams) {} void init(PUniqueId load_id, int64_t partition_id, int64_t index_id, int64_t tablet_id, int32_t segment_id); @@ -58,30 +56,10 @@ class StreamSinkFileWriter : public FileWriter { } private: - Status _stream_sender(butil::IOBuf buf) const { - for (auto stream : _streams) { - LOG(INFO) << "writer flushing, load_id: " << UniqueId(_load_id).to_string() - << ", index_id: " << _index_id << ", tablet_id: " << _tablet_id - << ", segment_id: " << _segment_id << ", stream id: " << stream - << ", buf size: " << buf.size(); - - RETURN_IF_ERROR(send_with_retry(stream, buf)); - } - return Status::OK(); - } - - void _append_header(); - -private: - PStreamHeader _header; - butil::IOBuf _buf; - - std::queue _pending_slices; - size_t _max_pending_bytes = config::brpc_streaming_client_batch_bytes; - size_t _pending_bytes; + template + Status _flush(); - int _sender_id; - std::vector _streams; + std::vector> _streams; PUniqueId _load_id; int64_t _partition_id; diff --git a/be/src/olap/bitmap_filter_predicate.h b/be/src/olap/bitmap_filter_predicate.h index 38133f3f3fa2509..f90e0e625b8611c 100644 --- a/be/src/olap/bitmap_filter_predicate.h +++ b/be/src/olap/bitmap_filter_predicate.h @@ -101,6 +101,7 @@ class BitmapFilterColumnPredicate : public ColumnPredicate { SpecificFilter* _specific_filter; // owned by _filter int get_filter_id() const override { return _filter->get_filter_id(); } + bool is_filter() const override { return true; } }; template diff --git a/be/src/olap/bloom_filter_predicate.h b/be/src/olap/bloom_filter_predicate.h index 885927d3f5665a8..d2816be9966dc36 100644 --- a/be/src/olap/bloom_filter_predicate.h +++ b/be/src/olap/bloom_filter_predicate.h @@ -155,7 +155,12 @@ class BloomFilterColumnPredicate : public ColumnPredicate { return info; } - int get_filter_id() const override { return _filter->get_filter_id(); } + int get_filter_id() const override { + int filter_id = _filter->get_filter_id(); + DCHECK(filter_id != -1); + return filter_id; + } + bool is_filter() const override { return true; } std::shared_ptr _filter; SpecificFilter* _specific_filter; // owned by _filter diff --git a/be/src/olap/column_predicate.h b/be/src/olap/column_predicate.h index cad253ac1ac17ed..b98156f5fb8a800 100644 --- a/be/src/olap/column_predicate.h +++ b/be/src/olap/column_predicate.h @@ -219,7 +219,8 @@ class ColumnPredicate { virtual void clone(ColumnPredicate** to) const { LOG(FATAL) << "clone not supported"; } virtual int get_filter_id() const { return -1; } - + // now InListPredicateBase BloomFilterColumnPredicate BitmapFilterColumnPredicate = true + virtual bool is_filter() const { return false; } PredicateFilterInfo get_filtered_info() const { return PredicateFilterInfo {static_cast(type()), _evaluated_rows - 1, _evaluated_rows - 1 - _passed_rows}; diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 70a9b5ada08aa71..b0940c45f17dd35 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -385,6 +385,7 @@ Status Compaction::do_compaction_impl(int64_t permits) { }); // now version in delete_predicate is deprecated if (!delete_predicate.in_predicates().empty() || + !delete_predicate.sub_predicates_v2().empty() || !delete_predicate.sub_predicates().empty()) { _output_rowset->rowset_meta()->set_delete_predicate(std::move(delete_predicate)); } @@ -668,8 +669,7 @@ Status Compaction::modify_rowsets(const Merger::Statistics* stats) { it.rowset_ids.insert(_output_rowset->rowset_id()); StorageEngine::instance()->txn_manager()->set_txn_related_delete_bitmap( it.partition_id, it.transaction_id, _tablet->tablet_id(), - _tablet->schema_hash(), _tablet->tablet_uid(), true, it.delete_bitmap, - it.rowset_ids); + _tablet->tablet_uid(), true, it.delete_bitmap, it.rowset_ids); } } @@ -699,6 +699,12 @@ Status Compaction::modify_rowsets(const Merger::Statistics* stats) { RETURN_IF_ERROR(_tablet->modify_rowsets(output_rowsets, _input_rowsets, true)); } + if (config::tablet_rowset_stale_sweep_by_size && + _tablet->tablet_meta()->all_stale_rs_metas().size() >= + config::tablet_rowset_stale_sweep_threshold_size) { + _tablet->delete_expired_stale_rowset(); + } + int64_t cur_max_version = 0; { std::shared_lock rlock(_tablet->get_header_lock()); diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index 39acf420c0129fc..bc488d06a4f4c75 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -17,6 +17,7 @@ #include "olap/data_dir.h" +#include #include #include #include @@ -25,6 +26,7 @@ #include // IWYU pragma: no_include #include // IWYU pragma: keep +#include #include #include #include @@ -43,6 +45,7 @@ #include "io/fs/local_file_system.h" #include "io/fs/path.h" #include "io/fs/remote_file_system.h" +#include "olap/delete_handler.h" #include "olap/olap_common.h" #include "olap/olap_define.h" #include "olap/olap_meta.h" @@ -259,14 +262,14 @@ Status DataDir::get_shard(uint64_t* shard) { } void DataDir::register_tablet(Tablet* tablet) { - TabletInfo tablet_info(tablet->tablet_id(), tablet->schema_hash(), tablet->tablet_uid()); + TabletInfo tablet_info(tablet->tablet_id(), tablet->tablet_uid()); std::lock_guard l(_mutex); _tablet_set.emplace(std::move(tablet_info)); } void DataDir::deregister_tablet(Tablet* tablet) { - TabletInfo tablet_info(tablet->tablet_id(), tablet->schema_hash(), tablet->tablet_uid()); + TabletInfo tablet_info(tablet->tablet_id(), tablet->tablet_uid()); std::lock_guard l(_mutex); _tablet_set.erase(tablet_info); @@ -366,7 +369,7 @@ Status DataDir::load() { std::vector dir_rowset_metas; LOG(INFO) << "begin loading rowset from meta"; - auto load_rowset_func = [&dir_rowset_metas, &local_fs = fs()]( + auto load_rowset_func = [&dir_rowset_metas, &local_fs = fs(), this]( TabletUid tablet_uid, RowsetId rowset_id, const std::string& meta_str) -> bool { RowsetMetaSharedPtr rowset_meta(new RowsetMeta()); @@ -379,6 +382,34 @@ Status DataDir::load() { if (rowset_meta->is_local()) { rowset_meta->set_fs(local_fs); } + if (rowset_meta->has_delete_predicate()) { + // copy the delete sub pred v1 to check then + auto orig_delete_sub_pred = rowset_meta->delete_predicate().sub_predicates(); + auto* delete_pred = rowset_meta->mutable_delete_pred_pb(); + + if ((!delete_pred->sub_predicates().empty() && + delete_pred->sub_predicates_v2().empty()) || + (!delete_pred->in_predicates().empty() && + delete_pred->in_predicates()[0].has_column_unique_id())) { + // convert pred and write only when delete sub pred v2 is not set or there is in list pred to be set column uid + DeleteHandler::convert_to_sub_pred_v2(delete_pred, rowset_meta->tablet_schema()); + LOG(INFO) << fmt::format( + "convert rowset with old delete pred: rowset_id={}, tablet_id={}", + rowset_id.to_string(), tablet_uid.to_string()); + CHECK_EQ(orig_delete_sub_pred.size(), delete_pred->sub_predicates().size()) + << "inconsistent sub predicate v1 after conversion"; + for (size_t i = 0; i < orig_delete_sub_pred.size(); ++i) { + CHECK_STREQ(orig_delete_sub_pred.Get(i).c_str(), + delete_pred->sub_predicates().Get(i).c_str()) + << "inconsistent sub predicate v1 after conversion"; + } + std::string result; + rowset_meta->serialize(&result); + std::string key = + ROWSET_PREFIX + tablet_uid.to_string() + "_" + rowset_id.to_string(); + _meta->put(META_COLUMN_FAMILY_INDEX, key, result); + } + } dir_rowset_metas.push_back(rowset_meta); return true; }; @@ -500,8 +531,8 @@ Status DataDir::load() { } Status commit_txn_status = _txn_manager->commit_txn( _meta, rowset_meta->partition_id(), rowset_meta->txn_id(), - rowset_meta->tablet_id(), rowset_meta->tablet_schema_hash(), - rowset_meta->tablet_uid(), rowset_meta->load_id(), rowset, true); + rowset_meta->tablet_id(), rowset_meta->tablet_uid(), rowset_meta->load_id(), + rowset, true); if (!commit_txn_status && !commit_txn_status.is()) { LOG(WARNING) << "failed to add committed rowset: " << rowset_meta->rowset_id() << " to tablet: " << rowset_meta->tablet_id() diff --git a/be/src/olap/delete_handler.cpp b/be/src/olap/delete_handler.cpp index 46bfe56cb788437..df0f6211aa3a24d 100644 --- a/be/src/olap/delete_handler.cpp +++ b/be/src/olap/delete_handler.cpp @@ -35,6 +35,7 @@ #include "olap/column_predicate.h" #include "olap/olap_common.h" #include "olap/predicate_creator.h" +#include "olap/tablet_schema.h" #include "olap/utils.h" using apache::thrift::ThriftDebugString; @@ -47,6 +48,8 @@ using std::regex_error; using std::regex_match; using std::smatch; +using ::google::protobuf::RepeatedPtrField; + namespace doris { using namespace ErrorCode; @@ -71,6 +74,9 @@ Status DeleteHandler::generate_delete_predicate(const TabletSchema& schema, for (const TCondition& condition : conditions) { if (condition.condition_values.size() > 1) { InPredicatePB* in_pred = del_pred->add_in_predicates(); + if (condition.__isset.column_unique_id) { + in_pred->set_column_unique_id(condition.column_unique_id); + } in_pred->set_column_name(condition.column_name); bool is_not_in = condition.condition_op == "!*="; in_pred->set_is_not_in(is_not_in); @@ -81,9 +87,18 @@ Status DeleteHandler::generate_delete_predicate(const TabletSchema& schema, LOG(INFO) << "store one sub-delete condition. condition name=" << in_pred->column_name() << "condition size=" << in_pred->values().size(); } else { - string condition_str = construct_sub_predicates(condition); - del_pred->add_sub_predicates(condition_str); - LOG(INFO) << "store one sub-delete condition. condition=" << condition_str; + // write sub predicate v1 for compactbility + del_pred->add_sub_predicates(construct_sub_predicate(condition)); + DeleteSubPredicatePB* sub_predicate = del_pred->add_sub_predicates_v2(); + if (condition.__isset.column_unique_id) { + sub_predicate->set_column_unique_id(condition.column_unique_id); + } + sub_predicate->set_column_name(condition.column_name); + sub_predicate->set_op(trans_op(condition.condition_op)); + sub_predicate->set_cond_value(condition.condition_values[0]); + LOG(INFO) << "store one sub-delete condition. condition=" + << fmt::format(" {} {} {}", condition.column_name, condition.condition_op, + condition.condition_values[0]); } } del_pred->set_version(-1); @@ -91,7 +106,27 @@ Status DeleteHandler::generate_delete_predicate(const TabletSchema& schema, return Status::OK(); } -std::string DeleteHandler::construct_sub_predicates(const TCondition& condition) { +void DeleteHandler::convert_to_sub_pred_v2(DeletePredicatePB* delete_pred, + TabletSchemaSPtr schema) { + if (!delete_pred->sub_predicates().empty() && delete_pred->sub_predicates_v2().empty()) { + for (const auto& condition_str : delete_pred->sub_predicates()) { + auto* sub_pred = delete_pred->add_sub_predicates_v2(); + TCondition condition; + parse_condition(condition_str, &condition); + sub_pred->set_column_unique_id(schema->column(condition.column_name).unique_id()); + sub_pred->set_column_name(condition.column_name); + sub_pred->set_op(condition.condition_op); + sub_pred->set_cond_value(condition.condition_values[0]); + } + } + + auto* in_pred_list = delete_pred->mutable_in_predicates(); + for (auto& in_pred : *in_pred_list) { + in_pred.set_column_unique_id(schema->column(in_pred.column_name()).unique_id()); + } +} + +std::string DeleteHandler::construct_sub_predicate(const TCondition& condition) { string op = condition.condition_op; if (op == "<") { op += "<"; @@ -112,6 +147,23 @@ std::string DeleteHandler::construct_sub_predicates(const TCondition& condition) return condition_str; } +std::string DeleteHandler::trans_op(const std::string& opt) { + std::string op = string(opt); + if (op == "<") { + op += "<"; + } else if (op == ">") { + op += ">"; + } + if ("IS" != op) { + if (op == "*=") { + op = "="; + } else if (op == "!*=") { + op = "!="; + } + } + return op; +} + bool DeleteHandler::is_condition_value_valid(const TabletColumn& column, const std::string& condition_op, const string& value_str) { @@ -200,7 +252,22 @@ Status DeleteHandler::check_condition_valid(const TabletSchema& schema, const TC return Status::OK(); } -bool DeleteHandler::_parse_condition(const std::string& condition_str, TCondition* condition) { +Status DeleteHandler::parse_condition(const DeleteSubPredicatePB& sub_cond, TCondition* condition) { + if (!sub_cond.has_column_name() || !sub_cond.has_op() || !sub_cond.has_cond_value()) { + return Status::Error( + "fail to parse condition. condition={} {} {}", sub_cond.column_name(), + sub_cond.op(), sub_cond.cond_value()); + } + if (sub_cond.has_column_unique_id()) { + condition->column_unique_id = sub_cond.column_unique_id(); + } + condition->column_name = sub_cond.column_name(); + condition->condition_op = sub_cond.op(); + condition->condition_values.push_back(sub_cond.cond_value()); + return Status::OK(); +} + +Status DeleteHandler::parse_condition(const std::string& condition_str, TCondition* condition) { bool matched = true; smatch what; @@ -227,7 +294,8 @@ bool DeleteHandler::_parse_condition(const std::string& condition_str, TConditio } if (!matched) { - return false; + return Status::Error("fail to sub condition. condition={}", + condition_str); } condition->column_name = what[1].str(); condition->condition_op = what[2].str(); @@ -237,11 +305,44 @@ bool DeleteHandler::_parse_condition(const std::string& condition_str, TConditio condition->condition_values.push_back(what[3].str()); } - return true; + return Status::OK(); +} + +template +Status DeleteHandler::_parse_column_pred(TabletSchemaSPtr complete_schema, + TabletSchemaSPtr delete_pred_related_schema, + const RepeatedPtrField& sub_pred_list, + DeleteConditions* delete_conditions) { + for (const auto& sub_predicate : sub_pred_list) { + TCondition condition; + RETURN_IF_ERROR(parse_condition(sub_predicate, &condition)); + int32_t col_unique_id = + delete_pred_related_schema->column(condition.column_name).unique_id(); + condition.__set_column_unique_id(col_unique_id); + const auto& column = complete_schema->column_by_uid(col_unique_id); + uint32_t index = complete_schema->field_index(col_unique_id); + auto* predicate = + parse_to_predicate(column, index, condition, _predicate_arena.get(), true); + if (predicate != nullptr) { + delete_conditions->column_predicate_vec.push_back(predicate); + } + } + return Status::OK(); } +template Status DeleteHandler::_parse_column_pred( + TabletSchemaSPtr complete_schema, TabletSchemaSPtr delete_pred_related_schema, + const ::google::protobuf::RepeatedPtrField& sub_pred_list, + DeleteConditions* delete_conditions); + +template Status DeleteHandler::_parse_column_pred( + TabletSchemaSPtr complete_schema, TabletSchemaSPtr delete_pred_related_schema, + const ::google::protobuf::RepeatedPtrField& sub_pred_list, + DeleteConditions* delete_conditions); + Status DeleteHandler::init(TabletSchemaSPtr tablet_schema, - const std::vector& delete_preds, int64_t version) { + const std::vector& delete_preds, int64_t version, + bool with_sub_pred_v2) { DCHECK(!_is_inited) << "reinitialize delete handler."; DCHECK(version >= 0) << "invalid parameters. version=" << version; _predicate_arena.reset(new vectorized::Arena()); @@ -256,26 +357,20 @@ Status DeleteHandler::init(TabletSchemaSPtr tablet_schema, auto& delete_condition = delete_pred->delete_predicate(); DeleteConditions temp; temp.filter_version = delete_pred->version().first; - for (const auto& sub_predicate : delete_condition.sub_predicates()) { - TCondition condition; - if (!_parse_condition(sub_predicate, &condition)) { - return Status::Error( - "fail to parse condition. condition={}", sub_predicate); - } - int32_t col_unique_id = - delete_pred_related_schema->column(condition.column_name).unique_id(); - const auto& column = tablet_schema->column_by_uid(col_unique_id); - uint32_t index = tablet_schema->field_index(col_unique_id); - auto predicate = - parse_to_predicate(column, index, condition, _predicate_arena.get(), true); - if (predicate != nullptr) { - temp.column_predicate_vec.push_back(predicate); - } + if (with_sub_pred_v2) { + RETURN_IF_ERROR(_parse_column_pred(tablet_schema, delete_pred_related_schema, + delete_condition.sub_predicates_v2(), &temp)); + } else { + // make it compatible with the former versions + RETURN_IF_ERROR(_parse_column_pred(tablet_schema, delete_pred_related_schema, + delete_condition.sub_predicates(), &temp)); } - for (const auto& in_predicate : delete_condition.in_predicates()) { TCondition condition; condition.__set_column_name(in_predicate.column_name()); + condition.__set_column_unique_id( + delete_pred_related_schema->column(condition.column_name).unique_id()); + if (in_predicate.is_not_in()) { condition.__set_condition_op("!*="); } else { diff --git a/be/src/olap/delete_handler.h b/be/src/olap/delete_handler.h index 7239f4326b74e99..429de58b4ed780c 100644 --- a/be/src/olap/delete_handler.h +++ b/be/src/olap/delete_handler.h @@ -62,8 +62,7 @@ class DeleteHandler { const std::vector& conditions, DeletePredicatePB* del_pred); - // construct sub condition from TCondition - static std::string construct_sub_predicates(const TCondition& condition); + static void convert_to_sub_pred_v2(DeletePredicatePB* delete_pred, TabletSchemaSPtr schema); private: // Validate the condition on the schema. @@ -78,6 +77,16 @@ class DeleteHandler { const std::string& condition_op, const std::string& value_str); + // construct sub condition from TCondition + static std::string construct_sub_predicate(const TCondition& condition); + + // make operators from FE adaptive to BE + [[nodiscard]] static std::string trans_op(const string& op); + + // extract 'column_name', 'op' and 'operands' to condition + static Status parse_condition(const DeleteSubPredicatePB& sub_cond, TCondition* condition); + static Status parse_condition(const std::string& condition_str, TCondition* condition); + public: DeleteHandler() = default; ~DeleteHandler() { finalize(); } @@ -85,17 +94,18 @@ class DeleteHandler { // Initialize DeleteHandler, use the delete conditions of this tablet whose version less than or equal to // 'version' to fill '_del_conds'. // NOTE: You should lock the tablet's header file before calling this function. - // // input: // * schema: tablet's schema, the delete conditions and data rows are in this schema // * version: maximum version + // * with_sub_pred_v2: whether to use delete sub predicate v2 (v2 is based on PB, v1 is based on condition string) // return: // * Status::Error(): input parameters are not valid // * Status::Error(): alloc memory failed Status init(TabletSchemaSPtr tablet_schema, - const std::vector& delete_conditions, int64_t version); + const std::vector& delete_conditions, int64_t version, + bool with_sub_pred_v2 = false); - bool empty() const { return _del_conds.empty(); } + [[nodiscard]] bool empty() const { return _del_conds.empty(); } // Release an instance of this class. void finalize(); @@ -106,8 +116,11 @@ class DeleteHandler { del_predicates_for_zone_map) const; private: - // Use regular expression to extract 'column_name', 'op' and 'operands' - bool _parse_condition(const std::string& condition_str, TCondition* condition); + template + Status _parse_column_pred( + TabletSchemaSPtr complete_schema, TabletSchemaSPtr delete_pred_related_schema, + const ::google::protobuf::RepeatedPtrField& sub_pred_list, + DeleteConditions* delete_conditions); bool _is_inited = false; // DeleteConditions in _del_conds are in 'OR' relationship diff --git a/be/src/olap/delta_writer_context.h b/be/src/olap/delta_writer_context.h index 680f2d0b6f85840..da506b10b52e978 100644 --- a/be/src/olap/delta_writer_context.h +++ b/be/src/olap/delta_writer_context.h @@ -28,7 +28,6 @@ namespace doris { class TupleDescriptor; class SlotDescriptor; class OlapTableSchemaParam; -class TabletSchema; struct WriteRequest { int64_t tablet_id; @@ -42,11 +41,6 @@ struct WriteRequest { bool is_high_priority = false; OlapTableSchemaParam* table_schema_param; int64_t index_id = 0; - // for DeltaWriterV2 - std::shared_ptr tablet_schema; - bool enable_unique_key_merge_on_write = false; - int sender_id = 0; - std::vector streams; }; } // namespace doris \ No newline at end of file diff --git a/be/src/olap/delta_writer_v2.cpp b/be/src/olap/delta_writer_v2.cpp index 6f42ae068e01d7f..ff1a341f5efd821 100644 --- a/be/src/olap/delta_writer_v2.cpp +++ b/be/src/olap/delta_writer_v2.cpp @@ -58,23 +58,27 @@ #include "util/stopwatch.hpp" #include "util/time.h" #include "vec/core/block.h" +#include "vec/sink/load_stream_stub.h" namespace doris { using namespace ErrorCode; -Status DeltaWriterV2::open(WriteRequest* req, DeltaWriterV2** writer, RuntimeProfile* profile) { - *writer = new DeltaWriterV2(req, StorageEngine::instance(), profile); +Status DeltaWriterV2::open(WriteRequest* req, + const std::vector>& streams, + DeltaWriterV2** writer, RuntimeProfile* profile) { + *writer = new DeltaWriterV2(req, streams, StorageEngine::instance(), profile); return Status::OK(); } -DeltaWriterV2::DeltaWriterV2(WriteRequest* req, StorageEngine* storage_engine, - RuntimeProfile* profile) +DeltaWriterV2::DeltaWriterV2(WriteRequest* req, + const std::vector>& streams, + StorageEngine* storage_engine, RuntimeProfile* profile) : _req(*req), _tablet_schema(new TabletSchema), _profile(profile->create_child(fmt::format("DeltaWriterV2 {}", _req.tablet_id), true, true)), _memtable_writer(new MemTableWriter(*req)), - _streams(req->streams) { + _streams(streams) { _init_profile(profile); } @@ -97,7 +101,8 @@ Status DeltaWriterV2::init() { return Status::OK(); } // build tablet schema in request level - _build_current_tablet_schema(_req.index_id, _req.table_schema_param, *_req.tablet_schema.get()); + _build_current_tablet_schema(_req.index_id, _req.table_schema_param, + *_streams[0]->tablet_schema(_req.index_id)); RowsetWriterContext context; context.txn_id = _req.txn_id; context.load_id = _req.load_id; @@ -112,17 +117,18 @@ Status DeltaWriterV2::init() { context.tablet_id = _req.tablet_id; context.partition_id = _req.partition_id; context.tablet_schema_hash = _req.schema_hash; - context.enable_unique_key_merge_on_write = _req.enable_unique_key_merge_on_write; + context.enable_unique_key_merge_on_write = _streams[0]->enable_unique_mow(_req.index_id); context.rowset_type = RowsetTypePB::BETA_ROWSET; context.rowset_id = StorageEngine::instance()->next_rowset_id(); context.data_dir = nullptr; - context.sender_id = _req.sender_id; _rowset_writer = std::make_shared(_streams); _rowset_writer->init(context); - _memtable_writer->init(_rowset_writer, _tablet_schema, _req.enable_unique_key_merge_on_write); + _memtable_writer->init(_rowset_writer, _tablet_schema, + _streams[0]->enable_unique_mow(_req.index_id)); ExecEnv::GetInstance()->memtable_memory_limiter()->register_writer(_memtable_writer); _is_init = true; + _streams.clear(); return Status::OK(); } diff --git a/be/src/olap/delta_writer_v2.h b/be/src/olap/delta_writer_v2.h index 0d781620351c70b..741d939fa8d9770 100644 --- a/be/src/olap/delta_writer_v2.h +++ b/be/src/olap/delta_writer_v2.h @@ -52,6 +52,7 @@ class TupleDescriptor; class SlotDescriptor; class OlapTableSchemaParam; class BetaRowsetWriterV2; +class LoadStreamStub; namespace vectorized { class Block; @@ -61,7 +62,9 @@ class Block; // This class is NOT thread-safe, external synchronization is required. class DeltaWriterV2 { public: - static Status open(WriteRequest* req, DeltaWriterV2** writer, RuntimeProfile* profile); + static Status open(WriteRequest* req, + const std::vector>& streams, + DeltaWriterV2** writer, RuntimeProfile* profile); ~DeltaWriterV2(); @@ -95,7 +98,8 @@ class DeltaWriterV2 { int64_t total_received_rows() const { return _total_received_rows; } private: - DeltaWriterV2(WriteRequest* req, StorageEngine* storage_engine, RuntimeProfile* profile); + DeltaWriterV2(WriteRequest* req, const std::vector>& streams, + StorageEngine* storage_engine, RuntimeProfile* profile); void _build_current_tablet_schema(int64_t index_id, const OlapTableSchemaParam* table_schema_param, @@ -122,7 +126,7 @@ class DeltaWriterV2 { std::shared_ptr _memtable_writer; MonotonicStopWatch _lock_watch; - std::vector _streams; + std::vector> _streams; }; } // namespace doris diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h index f4e432cf280491e..329c9b8dc0e269e 100644 --- a/be/src/olap/in_list_predicate.h +++ b/be/src/olap/in_list_predicate.h @@ -287,6 +287,8 @@ class InListPredicateBase : public ColumnPredicate { return new_size; } int get_filter_id() const override { return _values->get_filter_id(); } + bool is_filter() const override { return true; } + template void _evaluate_bit(const vectorized::IColumn& column, const uint16_t* sel, uint16_t size, bool* flags) const { diff --git a/be/src/olap/like_column_predicate.cpp b/be/src/olap/like_column_predicate.cpp index ef1374d4453d037..1d20104ee5a4c5f 100644 --- a/be/src/olap/like_column_predicate.cpp +++ b/be/src/olap/like_column_predicate.cpp @@ -22,29 +22,37 @@ #include "vec/columns/columns_number.h" #include "vec/columns/predicate_column.h" #include "vec/common/string_ref.h" +#include "vec/functions/like.h" namespace doris { -LikeColumnPredicate::LikeColumnPredicate(bool opposite, uint32_t column_id, - doris::FunctionContext* fn_ctx, doris::StringRef val) +template +LikeColumnPredicate::LikeColumnPredicate(bool opposite, uint32_t column_id, + doris::FunctionContext* fn_ctx, doris::StringRef val) : ColumnPredicate(column_id, opposite), pattern(val) { + static_assert(T == TYPE_VARCHAR || T == TYPE_CHAR || T == TYPE_STRING, + "LikeColumnPredicate only supports the following types: TYPE_VARCHAR, TYPE_CHAR, " + "TYPE_STRING"); _state = reinterpret_cast( fn_ctx->get_function_state(doris::FunctionContext::THREAD_LOCAL)); _state->search_state.clone(_like_state); } -void LikeColumnPredicate::evaluate_vec(const vectorized::IColumn& column, uint16_t size, - bool* flags) const { +template +void LikeColumnPredicate::evaluate_vec(const vectorized::IColumn& column, uint16_t size, + bool* flags) const { _evaluate_vec(column, size, flags); } -void LikeColumnPredicate::evaluate_and_vec(const vectorized::IColumn& column, uint16_t size, - bool* flags) const { +template +void LikeColumnPredicate::evaluate_and_vec(const vectorized::IColumn& column, uint16_t size, + bool* flags) const { _evaluate_vec(column, size, flags); } -uint16_t LikeColumnPredicate::evaluate(const vectorized::IColumn& column, uint16_t* sel, - uint16_t size) const { +template +uint16_t LikeColumnPredicate::evaluate(const vectorized::IColumn& column, uint16_t* sel, + uint16_t size) const { uint16_t new_size = 0; if (column.is_nullable()) { auto* nullable_col = vectorized::check_and_get_column(column); @@ -83,18 +91,18 @@ uint16_t LikeColumnPredicate::evaluate(const vectorized::IColumn& column, uint16 } } } else { - auto* str_col = - vectorized::check_and_get_column>( - nested_col); + auto* str_col = vectorized::check_and_get_column>( + nested_col); if (!nullable_col->has_null()) { vectorized::ColumnUInt8::Container res(size, 0); - (_state->predicate_like_function)( - const_cast(&_like_state), *str_col, pattern, - res, sel, size); for (uint16_t i = 0; i != size; i++) { uint16_t idx = sel[i]; sel[new_size] = idx; - new_size += _opposite ^ res[i]; + unsigned char flag = 0; + (_state->scalar_function)( + const_cast(&_like_state), + str_col->get_data_at(idx), pattern, &flag); + new_size += _opposite ^ flag; } } else { for (uint16_t i = 0; i != size; i++) { @@ -105,7 +113,7 @@ uint16_t LikeColumnPredicate::evaluate(const vectorized::IColumn& column, uint16 continue; } - StringRef cell_value = str_col->get_data()[idx]; + StringRef cell_value = str_col->get_data_at(idx); unsigned char flag = 0; (_state->scalar_function)( const_cast(&_like_state), @@ -130,21 +138,24 @@ uint16_t LikeColumnPredicate::evaluate(const vectorized::IColumn& column, uint16 new_size += _opposite ^ flag; } } else { - auto* str_col = - vectorized::check_and_get_column>( - column); + const vectorized::PredicateColumnType* str_col = + vectorized::check_and_get_column>(column); + vectorized::ColumnUInt8::Container res(size, 0); - (_state->predicate_like_function)( - const_cast(&_like_state), *str_col, pattern, res, - sel, size); for (uint16_t i = 0; i != size; i++) { uint16_t idx = sel[i]; sel[new_size] = idx; - new_size += _opposite ^ res[i]; + unsigned char flag = 0; + (_state->scalar_function)(const_cast(&_like_state), + str_col->get_data_at(idx), pattern, &flag); + new_size += _opposite ^ flag; } } } return new_size; } +template class LikeColumnPredicate; +template class LikeColumnPredicate; + } //namespace doris diff --git a/be/src/olap/like_column_predicate.h b/be/src/olap/like_column_predicate.h index f97ff464539e344..dca6cf218ac895d 100644 --- a/be/src/olap/like_column_predicate.h +++ b/be/src/olap/like_column_predicate.h @@ -47,6 +47,7 @@ namespace segment_v2 { class BitmapIndexIterator; } // namespace segment_v2 +template class LikeColumnPredicate : public ColumnPredicate { public: LikeColumnPredicate(bool opposite, uint32_t column_id, doris::FunctionContext* fn_ctx, diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index c9030d13177dcac..1a2f73f119571cd 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -73,14 +73,12 @@ struct DataDirInfoLessAvailability { }; struct TabletInfo { - TabletInfo(TTabletId in_tablet_id, TSchemaHash in_schema_hash, UniqueId in_uid) - : tablet_id(in_tablet_id), schema_hash(in_schema_hash), tablet_uid(in_uid) {} + TabletInfo(TTabletId in_tablet_id, UniqueId in_uid) + : tablet_id(in_tablet_id), tablet_uid(in_uid) {} bool operator<(const TabletInfo& right) const { if (tablet_id != right.tablet_id) { return tablet_id < right.tablet_id; - } else if (schema_hash != right.schema_hash) { - return schema_hash < right.schema_hash; } else { return tablet_uid < right.tablet_uid; } @@ -88,21 +86,19 @@ struct TabletInfo { std::string to_string() const { std::stringstream ss; - ss << tablet_id << "." << schema_hash << "." << tablet_uid.to_string(); + ss << tablet_id << "." << tablet_uid.to_string(); return ss.str(); } TTabletId tablet_id; - TSchemaHash schema_hash; UniqueId tablet_uid; }; struct TabletSize { - TabletSize(TTabletId in_tablet_id, TSchemaHash in_schema_hash, size_t in_tablet_size) - : tablet_id(in_tablet_id), schema_hash(in_schema_hash), tablet_size(in_tablet_size) {} + TabletSize(TTabletId in_tablet_id, size_t in_tablet_size) + : tablet_id(in_tablet_id), tablet_size(in_tablet_size) {} TTabletId tablet_id; - TSchemaHash schema_hash; size_t tablet_size; }; diff --git a/be/src/olap/olap_meta.h b/be/src/olap/olap_meta.h index 504a07f90f57a1c..7432c883b60dafc 100644 --- a/be/src/olap/olap_meta.h +++ b/be/src/olap/olap_meta.h @@ -17,6 +17,8 @@ #pragma once +#include + #include #include #include @@ -60,17 +62,21 @@ class OlapMeta final { Status iterate(const int column_family_index, const std::string& prefix, std::function const& func); + Status iterate(const int column_family_index, const std::string& seek_key, const std::string& prefix, std::function const& func); - std::string get_root_path() const { return _root_path; } + [[nodiscard]] std::string get_root_path() const { return _root_path; } rocksdb::ColumnFamilyHandle* get_handle(const int column_family_index) { return _handles[column_family_index].get(); } private: + Status get_iterator(const int column_family_index, const std::string& seek_key, + const std::string& prefix, rocksdb::Iterator*); + std::string _root_path; // keep order of _db && _handles, we need destroy _handles before _db std::unique_ptr> _db; diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp index e2840d29b44b453..6ac80b18daf2334 100644 --- a/be/src/olap/olap_server.cpp +++ b/be/src/olap/olap_server.cpp @@ -250,8 +250,7 @@ Status StorageEngine::start_bg_threads() { void StorageEngine::_cache_clean_callback() { int32_t interval = config::cache_prune_stale_interval; - while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit) { + while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))) { if (interval <= 0) { LOG(WARNING) << "config of cache clean interval is illegal: [" << interval << "], force set to 3600 "; @@ -259,6 +258,17 @@ void StorageEngine::_cache_clean_callback() { } CacheManager::instance()->for_each_cache_prune_stale(); + + // Dynamically modify the config to clear the cache, each time the disable cache will only be cleared once. + // TODO, Support page cache and other caches. + if (config::disable_segment_cache) { + if (!_clear_segment_cache) { + CacheManager::instance()->clear_once(CachePolicy::CacheType::SEGMENT_CACHE); + _clear_segment_cache = true; + } + } else { + _clear_segment_cache = false; + } } } @@ -279,8 +289,7 @@ void StorageEngine::_garbage_sweeper_thread_callback() { double usage = 1.0; // After the program starts, the first round of cleaning starts after min_interval. uint32_t curr_interval = min_interval; - while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(curr_interval)) && - !k_doris_exit) { + while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(curr_interval))) { // Function properties: // when usage < 0.6, ratio close to 1.(interval close to max_interval) // when usage at [0.6, 0.75], ratio is rapidly decreasing from 0.87 to 0.27. @@ -314,8 +323,7 @@ void StorageEngine::_disk_stat_monitor_thread_callback() { << ", force set to 1"; interval = 1; } - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))); } void StorageEngine::check_cumulative_compaction_config() { @@ -346,8 +354,7 @@ void StorageEngine::_unused_rowset_monitor_thread_callback() { << ", force set to 1"; interval = 1; } - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))); } void StorageEngine::_path_gc_thread_callback(DataDir* data_dir) { @@ -366,8 +373,7 @@ void StorageEngine::_path_gc_thread_callback(DataDir* data_dir) { << "will be forced set to half hour"; interval = 1800; // 0.5 hour } - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))); } void StorageEngine::_path_scan_thread_callback(DataDir* data_dir) { @@ -385,8 +391,7 @@ void StorageEngine::_path_scan_thread_callback(DataDir* data_dir) { << "will be forced set to one day"; interval = 24 * 3600; // one day } - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))); } void StorageEngine::_tablet_checkpoint_callback(const std::vector& data_dirs) { @@ -401,8 +406,7 @@ void StorageEngine::_tablet_checkpoint_callback(const std::vector& dat } } interval = config::generate_tablet_meta_checkpoint_tasks_interval_secs; - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))); } void StorageEngine::_tablet_path_check_callback() { @@ -421,8 +425,7 @@ void StorageEngine::_tablet_path_check_callback() { do { int32_t batch_size = config::tablet_path_check_batch_size; if (batch_size <= 0) { - if (_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit) { + if (_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))) { break; } continue; @@ -480,8 +483,7 @@ void StorageEngine::_tablet_path_check_callback() { } } - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))); } void StorageEngine::_adjust_compaction_thread_num() { @@ -637,8 +639,7 @@ void StorageEngine::_compaction_tasks_producer_callback() { } else { interval = 5000; // 5s to check disable_auto_compaction } - } while (!_stop_background_threads_latch.wait_for(std::chrono::milliseconds(interval)) && - !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::milliseconds(interval))); } void StorageEngine::_update_replica_infos_callback() { @@ -722,8 +723,7 @@ void StorageEngine::_update_replica_infos_callback() { start = end; } interval = config::update_replica_infos_interval_seconds; - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))); } Status StorageEngine::_submit_single_replica_compaction_task(TabletSharedPtr tablet, @@ -1088,14 +1088,12 @@ void StorageEngine::_cooldown_tasks_producer_callback() { LOG(INFO) << "failed to submit cooldown task"; } } - } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)) && - !k_doris_exit); + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))); } void StorageEngine::_remove_unused_remote_files_callback() { while (!_stop_background_threads_latch.wait_for( - std::chrono::seconds(config::remove_unused_remote_files_interval_sec)) && - !k_doris_exit) { + std::chrono::seconds(config::remove_unused_remote_files_interval_sec))) { LOG(INFO) << "begin to remove unused remote files"; Tablet::remove_unused_remote_files(); } @@ -1106,8 +1104,7 @@ void StorageEngine::_cold_data_compaction_producer_callback() { std::mutex tablet_submitted_mtx; while (!_stop_background_threads_latch.wait_for( - std::chrono::seconds(config::cold_data_compaction_interval_sec)) && - !k_doris_exit) { + std::chrono::seconds(config::cold_data_compaction_interval_sec))) { if (config::disable_auto_compaction || MemInfo::is_exceed_soft_mem_limit(GB_EXCHANGE_BYTE)) { continue; @@ -1216,7 +1213,7 @@ void StorageEngine::_cache_file_cleaner_tasks_producer_callback() { interval = 10; } bool stop = _stop_background_threads_latch.wait_for(std::chrono::seconds(interval)); - if (stop || k_doris_exit) { + if (stop) { break; } if (config::generate_cache_cleaner_task_interval_sec <= 0) { @@ -1264,8 +1261,7 @@ int64_t StorageEngine::get_pending_publish_min_version(int64_t tablet_id) { } void StorageEngine::_async_publish_callback() { - while (!_stop_background_threads_latch.wait_for(std::chrono::milliseconds(30)) && - !k_doris_exit) { + while (!_stop_background_threads_latch.wait_for(std::chrono::milliseconds(30))) { // tablet, publish_version std::vector> need_removed_tasks; { diff --git a/be/src/olap/page_cache.h b/be/src/olap/page_cache.h index 3f76546013d33ff..e1e48f285697deb 100644 --- a/be/src/olap/page_cache.h +++ b/be/src/olap/page_cache.h @@ -103,21 +103,24 @@ class StoragePageCache { class DataPageCache : public LRUCachePolicy { public: DataPageCache(size_t capacity, uint32_t num_shards) - : LRUCachePolicy("DataPageCache", capacity, LRUCacheType::SIZE, - config::data_page_cache_stale_sweep_time_sec, num_shards) {} + : LRUCachePolicy(CachePolicy::CacheType::DATA_PAGE_CACHE, capacity, + LRUCacheType::SIZE, config::data_page_cache_stale_sweep_time_sec, + num_shards) {} }; class IndexPageCache : public LRUCachePolicy { public: IndexPageCache(size_t capacity, uint32_t num_shards) - : LRUCachePolicy("IndexPageCache", capacity, LRUCacheType::SIZE, - config::index_page_cache_stale_sweep_time_sec, num_shards) {} + : LRUCachePolicy(CachePolicy::CacheType::INDEXPAGE_CACHE, capacity, + LRUCacheType::SIZE, config::index_page_cache_stale_sweep_time_sec, + num_shards) {} }; class PKIndexPageCache : public LRUCachePolicy { public: PKIndexPageCache(size_t capacity, uint32_t num_shards) - : LRUCachePolicy("PKIndexPageCache", capacity, LRUCacheType::SIZE, + : LRUCachePolicy(CachePolicy::CacheType::PK_INDEX_PAGE_CACHE, capacity, + LRUCacheType::SIZE, config::pk_index_page_cache_stale_sweep_time_sec, num_shards) {} }; diff --git a/be/src/olap/primary_key_index.cpp b/be/src/olap/primary_key_index.cpp index d03d0312bbca586..9451a8c0fdbd93d 100644 --- a/be/src/olap/primary_key_index.cpp +++ b/be/src/olap/primary_key_index.cpp @@ -84,6 +84,8 @@ Status PrimaryKeyIndexBuilder::finalize(segment_v2::PrimaryKeyIndexMetaPB* meta) RETURN_IF_ERROR( _bloom_filter_index_builder->finish(_file_writer, meta->mutable_bloom_filter_index())); _disk_size += _file_writer->bytes_appended() - start_size; + _primary_key_index_builder.reset(nullptr); + _bloom_filter_index_builder.reset(nullptr); return Status::OK(); } diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp index b86217d53731adb..17ead3f1552e265 100644 --- a/be/src/olap/reader.cpp +++ b/be/src/olap/reader.cpp @@ -501,7 +501,8 @@ Status TabletReader::_init_conditions_param(const ReaderParams& read_params) { // Function filter push down to storage engine auto is_like_predicate = [](ColumnPredicate* _pred) { - if (dynamic_cast(_pred)) { + if (dynamic_cast*>(_pred) != nullptr || + dynamic_cast*>(_pred) != nullptr) { return true; } @@ -594,10 +595,10 @@ ColumnPredicate* TabletReader::_parse_to_predicate(const FunctionFilter& functio if (index < 0) { return nullptr; } - - // currently only support like predicate - return new LikeColumnPredicate(function_filter._opposite, index, function_filter._fn_ctx, - function_filter._string_param); + const TabletColumn& column = _tablet_schema->column(index); + return create_column_predicate(index, std::make_shared(function_filter), + column.type(), _reader_context.runtime_state->be_exec_version(), + &column); } Status TabletReader::_init_delete_condition(const ReaderParams& read_params) { @@ -615,9 +616,17 @@ Status TabletReader::_init_delete_condition(const ReaderParams& read_params) { ((read_params.reader_type == ReaderType::READER_CUMULATIVE_COMPACTION && config::enable_delete_when_cumu_compaction)) || read_params.reader_type == ReaderType::READER_CHECKSUM); - + if (_filter_delete) { + // note(tsy): for compaction, keep delete sub pred v1 temporarily + return _delete_handler.init(_tablet_schema, read_params.delete_predicates, + read_params.version.second, false); + } + auto* runtime_state = read_params.runtime_state; + // note(tsy): for query, use session var to enable delete sub pred v2, for schema change, use v2 directly + bool enable_sub_pred_v2 = + runtime_state == nullptr ? true : runtime_state->enable_delete_sub_pred_v2(); return _delete_handler.init(_tablet_schema, read_params.delete_predicates, - read_params.version.second); + read_params.version.second, enable_sub_pred_v2); } Status TabletReader::init_reader_params_and_create_block( diff --git a/be/src/olap/rowset/beta_rowset_writer_v2.cpp b/be/src/olap/rowset/beta_rowset_writer_v2.cpp index d7a641b6e0961d4..a4bc32e32b01ef9 100644 --- a/be/src/olap/rowset/beta_rowset_writer_v2.cpp +++ b/be/src/olap/rowset/beta_rowset_writer_v2.cpp @@ -54,11 +54,12 @@ #include "util/time.h" #include "vec/common/schema_util.h" // LocalSchemaChangeRecorder #include "vec/core/block.h" +#include "vec/sink/load_stream_stub.h" namespace doris { using namespace ErrorCode; -BetaRowsetWriterV2::BetaRowsetWriterV2(const std::vector& streams) +BetaRowsetWriterV2::BetaRowsetWriterV2(const std::vector>& streams) : _next_segment_id(0), _num_segment(0), _num_rows_written(0), @@ -78,33 +79,20 @@ Status BetaRowsetWriterV2::init(const RowsetWriterContext& rowset_writer_context Status BetaRowsetWriterV2::create_file_writer(uint32_t segment_id, io::FileWriterPtr& file_writer) { auto partition_id = _context.partition_id; - auto sender_id = _context.sender_id; auto index_id = _context.index_id; auto tablet_id = _context.tablet_id; auto load_id = _context.load_id; - auto stream_writer = std::make_unique(sender_id, _streams); + auto stream_writer = std::make_unique(_streams); stream_writer->init(load_id, partition_id, index_id, tablet_id, segment_id); file_writer = std::move(stream_writer); return Status::OK(); } Status BetaRowsetWriterV2::add_segment(uint32_t segment_id, SegmentStatistics& segstat) { - butil::IOBuf buf; - PStreamHeader header; - header.set_src_id(_context.sender_id); - *header.mutable_load_id() = _context.load_id; - header.set_partition_id(_context.partition_id); - header.set_index_id(_context.index_id); - header.set_tablet_id(_context.tablet_id); - header.set_segment_id(segment_id); - header.set_opcode(doris::PStreamHeader::ADD_SEGMENT); - segstat.to_pb(header.mutable_segment_statistics()); - size_t header_len = header.ByteSizeLong(); - buf.append(reinterpret_cast(&header_len), sizeof(header_len)); - buf.append(header.SerializeAsString()); for (const auto& stream : _streams) { - io::StreamSinkFileWriter::send_with_retry(stream, buf); + RETURN_IF_ERROR(stream->add_segment(_context.partition_id, _context.index_id, + _context.tablet_id, segment_id, segstat)); } return Status::OK(); } diff --git a/be/src/olap/rowset/beta_rowset_writer_v2.h b/be/src/olap/rowset/beta_rowset_writer_v2.h index 919c12860745805..a9822722172b799 100644 --- a/be/src/olap/rowset/beta_rowset_writer_v2.h +++ b/be/src/olap/rowset/beta_rowset_writer_v2.h @@ -60,9 +60,11 @@ namespace vectorized::schema_util { class LocalSchemaChangeRecorder; } +class LoadStreamStub; + class BetaRowsetWriterV2 : public RowsetWriter { public: - BetaRowsetWriterV2(const std::vector& streams); + BetaRowsetWriterV2(const std::vector>& streams); ~BetaRowsetWriterV2() override; @@ -157,7 +159,7 @@ class BetaRowsetWriterV2 : public RowsetWriter { fmt::memory_buffer vlog_buffer; - std::vector _streams; + std::vector> _streams; int64_t _delete_bitmap_ns = 0; int64_t _segment_writer_ns = 0; diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index 360a3a02025a340..d4216b5a01420ed 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -265,6 +265,10 @@ class RowsetMeta { return rowset_meta_pb; } + inline DeletePredicatePB* mutable_delete_pred_pb() { + return _rowset_meta_pb.mutable_delete_predicate(); + } + bool is_singleton_delta() const { return has_version() && _rowset_meta_pb.start_version() == _rowset_meta_pb.end_version(); } diff --git a/be/src/olap/rowset/rowset_meta_manager.cpp b/be/src/olap/rowset/rowset_meta_manager.cpp index 5a0f968c7216b9a..2315d76c98abdcf 100644 --- a/be/src/olap/rowset/rowset_meta_manager.cpp +++ b/be/src/olap/rowset/rowset_meta_manager.cpp @@ -36,9 +36,6 @@ #include "olap/utils.h" namespace doris { -namespace { -const std::string ROWSET_PREFIX = "rst_"; -} // namespace using namespace ErrorCode; @@ -414,8 +411,7 @@ Status RowsetMetaManager::remove(OlapMeta* meta, TabletUid tablet_uid, const Row Status RowsetMetaManager::remove_binlog(OlapMeta* meta, const std::string& suffix) { return meta->remove(META_COLUMN_FAMILY_INDEX, - std::vector {kBinlogMetaPrefix.data() + suffix, - kBinlogDataPrefix.data() + suffix}); + {kBinlogMetaPrefix.data() + suffix, kBinlogDataPrefix.data() + suffix}); } Status RowsetMetaManager::ingest_binlog_metas(OlapMeta* meta, TabletUid tablet_uid, diff --git a/be/src/olap/rowset/rowset_meta_manager.h b/be/src/olap/rowset/rowset_meta_manager.h index 0c04cb686c5f317..6d7f09204004556 100644 --- a/be/src/olap/rowset/rowset_meta_manager.h +++ b/be/src/olap/rowset/rowset_meta_manager.h @@ -35,6 +35,9 @@ class RowsetMetaPB; } // namespace doris namespace doris { +namespace { +const std::string ROWSET_PREFIX = "rst_"; +} // namespace // Helper class for managing rowset meta of one root path. class RowsetMetaManager { diff --git a/be/src/olap/rowset/rowset_writer_context.h b/be/src/olap/rowset/rowset_writer_context.h index b8bfd1225c95458..cb78a1233ac72ce 100644 --- a/be/src/olap/rowset/rowset_writer_context.h +++ b/be/src/olap/rowset/rowset_writer_context.h @@ -45,7 +45,6 @@ struct RowsetWriterContext { rowset_type(BETA_ROWSET), rowset_state(PREPARED), version(Version(0, 0)), - sender_id(0), txn_id(0), tablet_uid(0, 0), segments_overlap(OVERLAP_UNKNOWN) { @@ -68,8 +67,6 @@ struct RowsetWriterContext { // properties for non-pending rowset Version version; - int sender_id; - // properties for pending rowset int64_t txn_id; PUniqueId load_id; diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_reader.cpp b/be/src/olap/rowset/segment_v2/bitmap_index_reader.cpp index 0e2c26ff30ae157..69dbf0535638dab 100644 --- a/be/src/olap/rowset/segment_v2/bitmap_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/bitmap_index_reader.cpp @@ -32,10 +32,19 @@ namespace doris { namespace segment_v2 { -Status BitmapIndexReader::load(bool use_page_cache, bool kept_in_memory) { - const IndexedColumnMetaPB& dict_meta = _bitmap_index_meta->dict_column(); - const IndexedColumnMetaPB& bitmap_meta = _bitmap_index_meta->bitmap_column(); - _has_null = _bitmap_index_meta->has_null(); +Status BitmapIndexReader::load(bool use_page_cache, bool kept_in_memory, + const BitmapIndexPB* index_meta) { + // TODO yyq: implement a new once flag to avoid status construct. + return _load_once.call([this, use_page_cache, kept_in_memory, index_meta] { + return _load(use_page_cache, kept_in_memory, index_meta); + }); +} + +Status BitmapIndexReader::_load(bool use_page_cache, bool kept_in_memory, + const BitmapIndexPB* index_meta) { + const IndexedColumnMetaPB& dict_meta = index_meta->dict_column(); + const IndexedColumnMetaPB& bitmap_meta = index_meta->bitmap_column(); + _has_null = index_meta->has_null(); _dict_column_reader.reset(new IndexedColumnReader(_file_reader, dict_meta)); _bitmap_column_reader.reset(new IndexedColumnReader(_file_reader, bitmap_meta)); diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_reader.h b/be/src/olap/rowset/segment_v2/bitmap_index_reader.h index c448c9f5cc95402..1ac16439891c37b 100644 --- a/be/src/olap/rowset/segment_v2/bitmap_index_reader.h +++ b/be/src/olap/rowset/segment_v2/bitmap_index_reader.h @@ -28,6 +28,7 @@ #include "olap/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/indexed_column_reader.h" #include "olap/types.h" +#include "util/once.h" namespace roaring { class Roaring; @@ -42,13 +43,11 @@ class BitmapIndexPB; class BitmapIndexReader { public: - explicit BitmapIndexReader(io::FileReaderSPtr file_reader, - const BitmapIndexPB* bitmap_index_meta) + explicit BitmapIndexReader(io::FileReaderSPtr file_reader) : _file_reader(std::move(file_reader)), - _type_info(get_scalar_type_info()), - _bitmap_index_meta(bitmap_index_meta) {} + _type_info(get_scalar_type_info()) {} - Status load(bool use_page_cache, bool kept_in_memory); + Status load(bool use_page_cache, bool kept_in_memory, const BitmapIndexPB*); // create a new column iterator. Client should delete returned iterator Status new_iterator(BitmapIndexIterator** iterator); @@ -57,13 +56,16 @@ class BitmapIndexReader { const TypeInfo* type_info() { return _type_info; } +private: + Status _load(bool use_page_cache, bool kept_in_memory, const BitmapIndexPB*); + private: friend class BitmapIndexIterator; io::FileReaderSPtr _file_reader; const TypeInfo* _type_info; - const BitmapIndexPB* _bitmap_index_meta; bool _has_null = false; + DorisCallOnce _load_once; std::unique_ptr _dict_column_reader; std::unique_ptr _bitmap_column_reader; }; diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp index e3ae352dd71d159..dd663b01759ac80 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp @@ -31,6 +31,13 @@ namespace doris { namespace segment_v2 { Status BloomFilterIndexReader::load(bool use_page_cache, bool kept_in_memory) { + // TODO yyq: implement a new once flag to avoid status construct. + return _load_once.call([this, use_page_cache, kept_in_memory] { + return _load(use_page_cache, kept_in_memory); + }); +} + +Status BloomFilterIndexReader::_load(bool use_page_cache, bool kept_in_memory) { const IndexedColumnMetaPB& bf_index_meta = _bloom_filter_index_meta->bloom_filter(); _bloom_filter_reader.reset(new IndexedColumnReader(_file_reader, bf_index_meta)); diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h index 452a7ea2aeb12f8..dc45e4f69227833 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h @@ -28,6 +28,7 @@ #include "olap/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/indexed_column_reader.h" #include "olap/types.h" +#include "util/once.h" namespace doris { @@ -52,10 +53,14 @@ class BloomFilterIndexReader { const TypeInfo* type_info() const { return _type_info; } +private: + Status _load(bool use_page_cache, bool kept_in_memory); + private: friend class BloomFilterIndexIterator; io::FileReaderSPtr _file_reader; + DorisCallOnce _load_once; const TypeInfo* _type_info; const BloomFilterIndexPB* _bloom_filter_index_meta; std::unique_ptr _bloom_filter_reader; diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp index 1f98b9032c27da2..48ac67a51f3fd0b 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/column_reader.cpp @@ -76,7 +76,7 @@ Status ColumnReader::create(const ColumnReaderOptions& opts, const ColumnMetaPB& if (is_scalar_type((FieldType)meta.type())) { std::unique_ptr reader_local( new ColumnReader(opts, meta, num_rows, file_reader)); - RETURN_IF_ERROR(reader_local->init()); + RETURN_IF_ERROR(reader_local->init(&meta)); *reader = std::move(reader_local); return Status::OK(); } else { @@ -178,37 +178,44 @@ Status ColumnReader::create(const ColumnReaderOptions& opts, const ColumnMetaPB& ColumnReader::ColumnReader(const ColumnReaderOptions& opts, const ColumnMetaPB& meta, uint64_t num_rows, io::FileReaderSPtr file_reader) - : _meta(meta), - _opts(opts), + : _opts(opts), _num_rows(num_rows), _file_reader(std::move(file_reader)), _dict_encoding_type(UNKNOWN_DICT_ENCODING), - _use_index_page_cache(!config::disable_storage_page_cache) {} + _use_index_page_cache(!config::disable_storage_page_cache) { + _meta_length = meta.length(); + _meta_type = (FieldType)meta.type(); + if (_meta_type == FieldType::OLAP_FIELD_TYPE_ARRAY) { + _meta_children_column_type = (FieldType)meta.children_columns(0).type(); + } + _meta_is_nullable = meta.is_nullable(); + _meta_dict_page = meta.dict_page(); + _meta_compression = meta.compression(); +} ColumnReader::~ColumnReader() = default; -Status ColumnReader::init() { - _type_info = get_type_info(&_meta); +Status ColumnReader::init(const ColumnMetaPB* meta) { + _type_info = get_type_info(meta); if (_type_info == nullptr) { - return Status::NotSupported("unsupported typeinfo, type={}", _meta.type()); + return Status::NotSupported("unsupported typeinfo, type={}", meta->type()); } - RETURN_IF_ERROR(EncodingInfo::get(_type_info.get(), _meta.encoding(), &_encoding_info)); + RETURN_IF_ERROR(EncodingInfo::get(_type_info.get(), meta->encoding(), &_encoding_info)); - for (int i = 0; i < _meta.indexes_size(); i++) { - auto& index_meta = _meta.indexes(i); + for (int i = 0; i < meta->indexes_size(); i++) { + auto& index_meta = meta->indexes(i); switch (index_meta.type()) { case ORDINAL_INDEX: _ordinal_index_meta = &index_meta.ordinal_index(); - _ordinal_index.reset( - new OrdinalIndexReader(_file_reader, _ordinal_index_meta, _num_rows)); + _ordinal_index.reset(new OrdinalIndexReader(_file_reader, _num_rows)); break; case ZONE_MAP_INDEX: _zone_map_index_meta = &index_meta.zone_map_index(); - _zone_map_index.reset(new ZoneMapIndexReader(_file_reader, _zone_map_index_meta)); + _zone_map_index.reset(new ZoneMapIndexReader(_file_reader)); break; case BITMAP_INDEX: _bitmap_index_meta = &index_meta.bitmap_index(); - _bitmap_index.reset(new BitmapIndexReader(_file_reader, _bitmap_index_meta)); + _bitmap_index.reset(new BitmapIndexReader(_file_reader)); break; case BLOOM_FILTER_INDEX: _bf_index_meta = &index_meta.bloom_filter_index(); @@ -223,7 +230,7 @@ Status ColumnReader::init() { // the item writer doesn't write any data and the corresponding ordinal index is empty. if (_ordinal_index_meta == nullptr && !is_empty()) { return Status::Corruption("Bad file {}: missing ordinal index for column {}", - _file_reader->path().native(), _meta.column_id()); + _file_reader->path().native(), meta->column_id()); } return Status::OK(); } @@ -279,8 +286,8 @@ Status ColumnReader::get_row_ranges_by_zone_map( Status ColumnReader::next_batch_of_zone_map(size_t* n, vectorized::MutableColumnPtr& dst) const { // TODO: this work to get min/max value seems should only do once FieldType type = _type_info->type(); - std::unique_ptr min_value(WrapperField::create_by_type(type, _meta.length())); - std::unique_ptr max_value(WrapperField::create_by_type(type, _meta.length())); + std::unique_ptr min_value(WrapperField::create_by_type(type, _meta_length)); + std::unique_ptr max_value(WrapperField::create_by_type(type, _meta_length)); _parse_zone_map_skip_null(_zone_map_index_meta->segment_zone_map(), min_value.get(), max_value.get()); @@ -320,8 +327,8 @@ bool ColumnReader::match_condition(const AndBlockColumnPredicate* col_predicates return true; } FieldType type = _type_info->type(); - std::unique_ptr min_value(WrapperField::create_by_type(type, _meta.length())); - std::unique_ptr max_value(WrapperField::create_by_type(type, _meta.length())); + std::unique_ptr min_value(WrapperField::create_by_type(type, _meta_length)); + std::unique_ptr max_value(WrapperField::create_by_type(type, _meta_length)); _parse_zone_map(_zone_map_index_meta->segment_zone_map(), min_value.get(), max_value.get()); return _zone_map_match_condition(_zone_map_index_meta->segment_zone_map(), min_value.get(), @@ -385,8 +392,8 @@ Status ColumnReader::_get_filtered_pages( FieldType type = _type_info->type(); const std::vector& zone_maps = _zone_map_index->page_zone_maps(); int32_t page_size = _zone_map_index->num_pages(); - std::unique_ptr min_value(WrapperField::create_by_type(type, _meta.length())); - std::unique_ptr max_value(WrapperField::create_by_type(type, _meta.length())); + std::unique_ptr min_value(WrapperField::create_by_type(type, _meta_length)); + std::unique_ptr max_value(WrapperField::create_by_type(type, _meta_length)); for (int32_t i = 0; i < page_size; ++i) { if (zone_maps[i].pass_all()) { page_indexes->push_back(i); @@ -465,25 +472,19 @@ Status ColumnReader::get_row_ranges_by_bloom_filter(const AndBlockColumnPredicat Status ColumnReader::_load_ordinal_index(bool use_page_cache, bool kept_in_memory) { DCHECK(_ordinal_index_meta != nullptr); - return _load_ordinal_index_once.call([this, use_page_cache, kept_in_memory] { - return _ordinal_index->load(use_page_cache, kept_in_memory); - }); + return _ordinal_index->load(use_page_cache, kept_in_memory, _ordinal_index_meta); } Status ColumnReader::_load_zone_map_index(bool use_page_cache, bool kept_in_memory) { if (_zone_map_index_meta != nullptr) { - return _load_zone_map_index_once.call([this, use_page_cache, kept_in_memory] { - return _zone_map_index->load(use_page_cache, kept_in_memory); - }); + return _zone_map_index->load(use_page_cache, kept_in_memory, _zone_map_index_meta); } return Status::OK(); } Status ColumnReader::_load_bitmap_index(bool use_page_cache, bool kept_in_memory) { if (_bitmap_index_meta != nullptr) { - return _load_bitmap_index_once.call([this, use_page_cache, kept_in_memory] { - return _bitmap_index->load(use_page_cache, kept_in_memory); - }); + return _bitmap_index->load(use_page_cache, kept_in_memory, _bitmap_index_meta); } return Status::OK(); } @@ -499,8 +500,8 @@ Status ColumnReader::_load_inverted_index_index(const TabletIndex* index_meta) { InvertedIndexParserType parser_type = get_inverted_index_parser_type_from_string( get_parser_string_from_properties(index_meta->properties())); FieldType type; - if ((FieldType)_meta.type() == FieldType::OLAP_FIELD_TYPE_ARRAY) { - type = (FieldType)_meta.children_columns(0).type(); + if (_meta_type == FieldType::OLAP_FIELD_TYPE_ARRAY) { + type = _meta_children_column_type; } else { type = _type_info->type(); } @@ -526,9 +527,7 @@ Status ColumnReader::_load_inverted_index_index(const TabletIndex* index_meta) { Status ColumnReader::_load_bloom_filter_index(bool use_page_cache, bool kept_in_memory) { if (_bf_index_meta != nullptr) { - return _load_bloom_filter_index_once.call([this, use_page_cache, kept_in_memory] { - return _bloom_filter_index->load(use_page_cache, kept_in_memory); - }); + return _bloom_filter_index->load(use_page_cache, kept_in_memory); } return Status::OK(); } @@ -556,11 +555,11 @@ Status ColumnReader::new_iterator(ColumnIterator** iterator) { *iterator = new EmptyFileColumnIterator(); return Status::OK(); } - if (is_scalar_type((FieldType)_meta.type())) { + if (is_scalar_type((FieldType)_meta_type)) { *iterator = new FileColumnIterator(this); return Status::OK(); } else { - auto type = (FieldType)_meta.type(); + auto type = (FieldType)_meta_type; switch (type) { case FieldType::OLAP_FIELD_TYPE_STRUCT: { std::vector sub_column_iterators; @@ -830,6 +829,7 @@ Status OffsetFileColumnIterator::_calculate_offsets( auto& offsets_data = column_offsets.get_data(); ordinal_t first_column_offset = offsets_data[start - 1]; // -1 is valid ordinal_t first_storage_offset = offsets_data[start]; + DCHECK(next_storage_offset >= first_storage_offset); for (ssize_t i = start; i < offsets_data.size() - 1; ++i) { offsets_data[i] = first_column_offset + (offsets_data[i + 1] - first_storage_offset); } diff --git a/be/src/olap/rowset/segment_v2/column_reader.h b/be/src/olap/rowset/segment_v2/column_reader.h index fb212ef33d249ec..52b6092f9217334 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.h +++ b/be/src/olap/rowset/segment_v2/column_reader.h @@ -131,7 +131,7 @@ class ColumnReader { PageHandle* handle, Slice* page_body, PageFooterPB* footer, BlockCompressionCodec* codec) const; - bool is_nullable() const { return _meta.is_nullable(); } + bool is_nullable() const { return _meta_is_nullable; } const EncodingInfo* encoding_info() const { return _encoding_info; } @@ -165,11 +165,11 @@ class ColumnReader { Status get_row_ranges_by_bloom_filter(const AndBlockColumnPredicate* col_predicates, RowRanges* row_ranges); - PagePointer get_dict_page_pointer() const { return _meta.dict_page(); } + PagePointer get_dict_page_pointer() const { return _meta_dict_page; } bool is_empty() const { return _num_rows == 0; } - CompressionTypePB get_compression() const { return _meta.compression(); } + CompressionTypePB get_compression() const { return _meta_compression; } uint64_t num_rows() const { return _num_rows; } @@ -187,7 +187,7 @@ class ColumnReader { private: ColumnReader(const ColumnReaderOptions& opts, const ColumnMetaPB& meta, uint64_t num_rows, io::FileReaderSPtr file_reader); - Status init(); + Status init(const ColumnMetaPB* meta); // Read column inverted indexes into memory // May be called multiple times, subsequent calls will no op. @@ -220,7 +220,13 @@ class ColumnReader { Status _calculate_row_ranges(const std::vector& page_indexes, RowRanges* row_ranges); private: - ColumnMetaPB _meta; + int64_t _meta_length; + FieldType _meta_type; + FieldType _meta_children_column_type; + bool _meta_is_nullable; + PagePointer _meta_dict_page; + CompressionTypePB _meta_compression; + ColumnReaderOptions _opts; uint64_t _num_rows; @@ -247,15 +253,9 @@ class ColumnReader { std::unique_ptr _bitmap_index; std::shared_ptr _inverted_index; std::unique_ptr _bloom_filter_index; - DorisCallOnce _load_zone_map_index_once; - DorisCallOnce _load_ordinal_index_once; - DorisCallOnce _load_bitmap_index_once; - DorisCallOnce _load_bloom_filter_index_once; - DorisCallOnce _load_inverted_index_once; std::vector> _sub_readers; - std::once_flag _set_dict_encoding_type_flag; DorisCallOnce _set_dict_encoding_type_once; }; diff --git a/be/src/olap/rowset/segment_v2/column_writer.cpp b/be/src/olap/rowset/segment_v2/column_writer.cpp index 2f27df779b60489..ec2baa10f0bb8d4 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.cpp +++ b/be/src/olap/rowset/segment_v2/column_writer.cpp @@ -905,49 +905,34 @@ size_t ArrayColumnWriter::get_inverted_index_size() { return 0; } -// Now we can only write data one by one. +// batch append data for array Status ArrayColumnWriter::append_data(const uint8_t** ptr, size_t num_rows) { - size_t remaining = num_rows; - const auto* col_cursor = reinterpret_cast(*ptr); - while (remaining > 0) { - // TODO llj: bulk write - size_t num_written = 1; - ordinal_t next_item_ordinal = _item_writer->get_next_rowid(); - RETURN_IF_ERROR(_offset_writer->append_data_in_current_page( - reinterpret_cast(&next_item_ordinal), &num_written)); - if (num_written < - 1) { // page is full, write first item offset and update current length page's start ordinal - RETURN_IF_ERROR(_offset_writer->finish_current_page()); - } else { - // write child item. - if (_item_writer->is_nullable()) { - auto* item_data_ptr = const_cast(col_cursor)->mutable_data(); - for (size_t i = 0; i < col_cursor->length(); ++i) { - RETURN_IF_ERROR(_item_writer->append(col_cursor->is_null_at(i), item_data_ptr)); - item_data_ptr = (uint8_t*)item_data_ptr + _item_writer->get_field()->size(); - } - } else { - const void* data = col_cursor->data(); - RETURN_IF_ERROR(_item_writer->append_data(reinterpret_cast(&data), - col_cursor->length())); - } - if (_opts.inverted_index) { - auto writer = dynamic_cast(_item_writer.get()); - if (writer != nullptr) { - //NOTE: use array field name as index field, but item_writer size should be used when moving item_data_ptr - RETURN_IF_ERROR(_inverted_index_builder->add_array_values( - _item_writer->get_field()->size(), col_cursor, 1)); - } + // data_ptr contains + // [size, offset_ptr, item_data_ptr, item_nullmap_ptr] + auto data_ptr = reinterpret_cast(*ptr); + // total number length + size_t element_cnt = size_t((unsigned long)(*data_ptr)); + auto offset_data = *(data_ptr + 1); + const uint8_t* offsets_ptr = (const uint8_t*)offset_data; + + if (element_cnt > 0) { + auto data = *(data_ptr + 2); + auto nested_null_map = *(data_ptr + 3); + RETURN_IF_ERROR(_item_writer->append(reinterpret_cast(nested_null_map), + reinterpret_cast(data), element_cnt)); + if (_opts.inverted_index) { + auto writer = dynamic_cast(_item_writer.get()); + // now only support nested type is scala + if (writer != nullptr) { + //NOTE: use array field name as index field, but item_writer size should be used when moving item_data_ptr + _inverted_index_builder->add_array_values( + _item_writer->get_field()->size(), reinterpret_cast(data), + reinterpret_cast(nested_null_map), offsets_ptr, num_rows); } } - remaining -= num_written; - col_cursor += num_written; - *ptr += num_written * sizeof(CollectionValue); } - if (is_nullable()) { - return write_null_column(num_rows, false); - } + RETURN_IF_ERROR(_offset_writer->append_data(&offsets_ptr, num_rows)); return Status::OK(); } @@ -957,6 +942,14 @@ uint64_t ArrayColumnWriter::estimate_buffer_size() { _item_writer->estimate_buffer_size(); } +Status ArrayColumnWriter::append_nullable(const uint8_t* null_map, const uint8_t** ptr, + size_t num_rows) { + RETURN_IF_ERROR(append_data(ptr, num_rows)); + if (is_nullable()) { + RETURN_IF_ERROR(_null_writer->append_data(&null_map, num_rows)); + } + return Status::OK(); +} Status ArrayColumnWriter::finish() { RETURN_IF_ERROR(_offset_writer->finish()); if (is_nullable()) { diff --git a/be/src/olap/rowset/segment_v2/column_writer.h b/be/src/olap/rowset/segment_v2/column_writer.h index 120432c3bb51b9a..b5aabd4e3ab8713 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.h +++ b/be/src/olap/rowset/segment_v2/column_writer.h @@ -345,6 +345,7 @@ class ArrayColumnWriter final : public ColumnWriter, public FlushPageCallback { Status write_data() override; Status write_ordinal_index() override; Status append_nulls(size_t num_rows) override; + Status append_nullable(const uint8_t* null_map, const uint8_t** ptr, size_t num_rows) override; Status finish_current_page() override; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp b/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp index f3c68984ebbfa70..055365cf31fb891 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp @@ -62,7 +62,7 @@ void InvertedIndexSearcherCache::create_global_instance(size_t capacity, uint32_ } InvertedIndexSearcherCache::InvertedIndexSearcherCache(size_t capacity, uint32_t num_shards) - : LRUCachePolicy("InvertedIndexSearcherCache", + : LRUCachePolicy(CachePolicy::CacheType::INVERTEDINDEX_SEARCHER_CACHE, config::inverted_index_cache_stale_sweep_time_sec), _mem_tracker(std::make_unique("InvertedIndexSearcherCache")) { SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); diff --git a/be/src/olap/rowset/segment_v2/inverted_index_cache.h b/be/src/olap/rowset/segment_v2/inverted_index_cache.h index 9f368eca0c55930..388ee02ee96aa85 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_cache.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_cache.h @@ -237,8 +237,9 @@ class InvertedIndexQueryCache : public LRUCachePolicy { InvertedIndexQueryCache() = delete; InvertedIndexQueryCache(size_t capacity, uint32_t num_shards) - : LRUCachePolicy("InvertedIndexQueryCache", capacity, LRUCacheType::SIZE, - config::inverted_index_cache_stale_sweep_time_sec, num_shards) {} + : LRUCachePolicy(CachePolicy::CacheType::INVERTEDINDEX_QUERY_CACHE, capacity, + LRUCacheType::SIZE, config::inverted_index_cache_stale_sweep_time_sec, + num_shards) {} bool lookup(const CacheKey& key, InvertedIndexQueryCacheHandle* handle); diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp index c05c766c47cf1ab..c2a622699a51be6 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp @@ -213,9 +213,8 @@ Status InvertedIndexReader::read_null_bitmap(InvertedIndexQueryCacheHandle* cach if (owned_dir) { FINALLY_FINALIZE_INPUT(dir); } - LOG(WARNING) << "Inverted index read null bitmap error occurred: " << e.what(); return Status::Error( - "Inverted index read null bitmap error occurred"); + "Inverted index read null bitmap error occurred, reason={}", e.what()); } return Status::OK(); diff --git a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp index e779170d0316a79..2194d349a9c1395 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp @@ -282,6 +282,60 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { return Status::OK(); } + Status add_array_values(size_t field_size, const void* value_ptr, const uint8_t* null_map, + const uint8_t* offsets_ptr, size_t count) override { + if (count == 0) { + // no values to add inverted index + return Status::OK(); + } + auto offsets = reinterpret_cast(offsets_ptr); + if constexpr (field_is_slice_type(field_type)) { + if (_field == nullptr || _index_writer == nullptr) { + LOG(ERROR) << "field or index writer is null in inverted index writer."; + return Status::InternalError( + "field or index writer is null in inverted index writer"); + } + for (int i = 0; i < count; ++i) { + // offsets[i+1] is now row element count + std::vector strings; + // [0, 3, 6] + // [10,20,30] [20,30,40], [30,40,50] + auto start_off = offsets[i]; + auto end_off = offsets[i + 1]; + for (auto j = start_off; j < end_off; ++j) { + if (null_map[j] == 1) { + continue; + } + auto* v = (Slice*)((const uint8_t*)value_ptr + j * field_size); + strings.emplace_back(std::string(v->get_data(), v->get_size())); + } + + auto value = join(strings, " "); + new_fulltext_field(value.c_str(), value.length()); + _rid++; + _index_writer->addDocument(_doc.get()); + } + } else if constexpr (field_is_numeric_type(field_type)) { + for (int i = 0; i < count; ++i) { + auto start_off = offsets[i]; + auto end_off = offsets[i + 1]; + for (size_t j = start_off; j < end_off; ++j) { + if (null_map[j] == 1) { + continue; + } + const CppType* p = &reinterpret_cast(value_ptr)[j]; + std::string new_value; + size_t value_length = sizeof(CppType); + + _value_key_coder->full_encode_ascending(p, &new_value); + _bkd_writer->add((const uint8_t*)new_value.c_str(), value_length, _rid); + } + _row_ids_seen_for_bkd++; + _rid++; + } + } + return Status::OK(); + } Status add_array_values(size_t field_size, const CollectionValue* values, size_t count) override { if constexpr (field_is_slice_type(field_type)) { diff --git a/be/src/olap/rowset/segment_v2/inverted_index_writer.h b/be/src/olap/rowset/segment_v2/inverted_index_writer.h index 6099f1d37327bf7..f9bce4064086768 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.h @@ -48,6 +48,10 @@ class InvertedIndexColumnWriter { virtual Status add_array_values(size_t field_size, const CollectionValue* values, size_t count) = 0; + virtual Status add_array_values(size_t field_size, const void* value_ptr, + const uint8_t* null_map, const uint8_t* offsets_ptr, + size_t count) = 0; + virtual Status add_nulls(uint32_t count) = 0; virtual Status finish() = 0; diff --git a/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp b/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp index d977e29f1152a61..d3d6d9cb9cd721a 100644 --- a/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp +++ b/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp @@ -67,19 +67,28 @@ Status OrdinalIndexWriter::finish(io::FileWriter* file_writer, ColumnIndexMetaPB return Status::OK(); } -Status OrdinalIndexReader::load(bool use_page_cache, bool kept_in_memory) { - if (_index_meta->root_page().is_root_data_page()) { +Status OrdinalIndexReader::load(bool use_page_cache, bool kept_in_memory, + const OrdinalIndexPB* index_meta) { + // TODO yyq: implement a new once flag to avoid status construct. + return _load_once.call([this, use_page_cache, kept_in_memory, index_meta] { + return _load(use_page_cache, kept_in_memory, index_meta); + }); +} + +Status OrdinalIndexReader::_load(bool use_page_cache, bool kept_in_memory, + const OrdinalIndexPB* index_meta) { + if (index_meta->root_page().is_root_data_page()) { // only one data page, no index page _num_pages = 1; _ordinals.push_back(0); _ordinals.push_back(_num_values); - _pages.emplace_back(_index_meta->root_page().root_page()); + _pages.emplace_back(index_meta->root_page().root_page()); return Status::OK(); } // need to read index page PageReadOptions opts; opts.file_reader = _file_reader.get(); - opts.page_pointer = PagePointer(_index_meta->root_page().root_page()); + opts.page_pointer = PagePointer(index_meta->root_page().root_page()); opts.codec = nullptr; // ordinal index page uses NO_COMPRESSION right now OlapReaderStatistics tmp_stats; opts.stats = &tmp_stats; diff --git a/be/src/olap/rowset/segment_v2/ordinal_page_index.h b/be/src/olap/rowset/segment_v2/ordinal_page_index.h index 17dba9e675cd63f..84a25c78a73a5f3 100644 --- a/be/src/olap/rowset/segment_v2/ordinal_page_index.h +++ b/be/src/olap/rowset/segment_v2/ordinal_page_index.h @@ -30,6 +30,7 @@ #include "olap/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/index_page.h" #include "olap/rowset/segment_v2/page_pointer.h" +#include "util/once.h" namespace doris { @@ -65,14 +66,11 @@ class OrdinalPageIndexIterator; class OrdinalIndexReader { public: - explicit OrdinalIndexReader(io::FileReaderSPtr file_reader, const OrdinalIndexPB* index_meta, - ordinal_t num_values) - : _file_reader(std::move(file_reader)), - _index_meta(index_meta), - _num_values(num_values) {} + explicit OrdinalIndexReader(io::FileReaderSPtr file_reader, ordinal_t num_values) + : _file_reader(std::move(file_reader)), _num_values(num_values) {} // load and parse the index page into memory - Status load(bool use_page_cache, bool kept_in_memory); + Status load(bool use_page_cache, bool kept_in_memory, const OrdinalIndexPB* index_meta); // the returned iter points to the largest element which is less than `ordinal`, // or points to the first element if all elements are greater than `ordinal`, @@ -89,11 +87,15 @@ class OrdinalIndexReader { // for test int32_t num_data_pages() const { return _num_pages; } +private: + Status _load(bool use_page_cache, bool kept_in_memory, const OrdinalIndexPB* index_meta); + private: friend OrdinalPageIndexIterator; io::FileReaderSPtr _file_reader; - const OrdinalIndexPB* _index_meta; + DorisCallOnce _load_once; + // total number of values (including NULLs) in the indexed column, // equals to 1 + 'last ordinal of last data pages' ordinal_t _num_values; diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp index c6d472d0358baeb..8a11cd004973e04 100644 --- a/be/src/olap/rowset/segment_v2/segment.cpp +++ b/be/src/olap/rowset/segment_v2/segment.cpp @@ -367,14 +367,14 @@ Status Segment::lookup_row_key(const Slice& key, bool with_seq_col, RowLocation* DCHECK(_pk_index_reader != nullptr); if (!_pk_index_reader->check_present(key_without_seq)) { - return Status::NotFound("Can't find key in the segment"); + return Status::Error("Can't find key in the segment"); } bool exact_match = false; std::unique_ptr index_iterator; RETURN_IF_ERROR(_pk_index_reader->new_iterator(&index_iterator)); RETURN_IF_ERROR(index_iterator->seek_at_or_after(&key_without_seq, &exact_match)); if (!has_seq_col && !exact_match) { - return Status::NotFound("Can't find key in the segment"); + return Status::Error("Can't find key in the segment"); } row_location->row_id = index_iterator->get_current_ordinal(); row_location->segment_id = _segment_id; @@ -396,7 +396,7 @@ Status Segment::lookup_row_key(const Slice& key, bool with_seq_col, RowLocation* // compare key if (key_without_seq.compare(sought_key_without_seq) != 0) { - return Status::NotFound("Can't find key in the segment"); + return Status::Error("Can't find key in the segment"); } if (!with_seq_col) { @@ -409,7 +409,8 @@ Status Segment::lookup_row_key(const Slice& key, bool with_seq_col, RowLocation* Slice previous_sequence_id = Slice( sought_key.get_data() + sought_key_without_seq.get_size() + 1, seq_col_length - 1); if (sequence_id.compare(previous_sequence_id) < 0) { - return Status::AlreadyExist("key with higher sequence id exists"); + return Status::Error( + "key with higher sequence id exists"); } } diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index 87b8014117d003e..d5e0321455c6bb5 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -23,10 +23,12 @@ #include #include +#include #include #include #include #include +#include // IWYU pragma: no_include #include "common/compiler_util.h" // IWYU pragma: keep @@ -198,6 +200,9 @@ class SegmentIterator::BackwardBitmapRangeIterator : public SegmentIterator::Bit SegmentIterator::SegmentIterator(std::shared_ptr segment, SchemaSPtr schema) : _segment(std::move(segment)), _schema(schema), + _column_iterators(_schema->num_columns()), + _bitmap_index_iterators(_schema->num_columns()), + _inverted_index_iterators(_schema->num_columns()), _cur_rowid(0), _lazy_materialization_read(false), _lazy_inited(false), @@ -368,16 +373,15 @@ Status SegmentIterator::_prepare_seek(const StorageReadOptions::KeyRange& key_ra // create used column iterator for (auto cid : _seek_schema->column_ids()) { - int32_t unique_id = _opts.tablet_schema->column(cid).unique_id(); - if (_column_iterators.count(unique_id) < 1) { + if (_column_iterators[cid] == nullptr) { RETURN_IF_ERROR(_segment->new_column_iterator(_opts.tablet_schema->column(cid), - &_column_iterators[unique_id])); + &_column_iterators[cid])); ColumnIteratorOptions iter_opts; iter_opts.stats = _opts.stats; iter_opts.use_page_cache = _opts.use_page_cache; iter_opts.file_reader = _file_reader.get(); iter_opts.io_ctx = _opts.io_ctx; - RETURN_IF_ERROR(_column_iterators[unique_id]->init(iter_opts)); + RETURN_IF_ERROR(_column_iterators[cid]->init(iter_opts)); } } @@ -440,8 +444,7 @@ Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row // get row ranges by bf index of this column, RowRanges column_bf_row_ranges = RowRanges::create_single(num_rows()); DCHECK(_opts.col_id_to_predicates.count(cid) > 0); - uint32_t unique_cid = _schema->unique_id(cid); - RETURN_IF_ERROR(_column_iterators[unique_cid]->get_row_ranges_by_bloom_filter( + RETURN_IF_ERROR(_column_iterators[cid]->get_row_ranges_by_bloom_filter( _opts.col_id_to_predicates.at(cid).get(), &column_bf_row_ranges)); RowRanges::ranges_intersection(bf_row_ranges, column_bf_row_ranges, &bf_row_ranges); } @@ -456,7 +459,7 @@ Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row // get row ranges by zone map of this column, RowRanges column_row_ranges = RowRanges::create_single(num_rows()); DCHECK(_opts.col_id_to_predicates.count(cid) > 0); - RETURN_IF_ERROR(_column_iterators[_schema->unique_id(cid)]->get_row_ranges_by_zone_map( + RETURN_IF_ERROR(_column_iterators[cid]->get_row_ranges_by_zone_map( _opts.col_id_to_predicates.at(cid).get(), _opts.del_predicates_for_zone_map.count(cid) > 0 ? &(_opts.del_predicates_for_zone_map.at(cid)) @@ -472,14 +475,14 @@ Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row auto query_ctx = _opts.runtime_state->get_query_ctx(); runtime_predicate = query_ctx->get_runtime_predicate().get_predictate(); if (runtime_predicate) { - int32_t cid = _opts.tablet_schema->column(runtime_predicate->column_id()).unique_id(); AndBlockColumnPredicate and_predicate; auto single_predicate = new SingleColumnBlockPredicate(runtime_predicate.get()); and_predicate.add_column_predicate(single_predicate); RowRanges column_rp_row_ranges = RowRanges::create_single(num_rows()); - RETURN_IF_ERROR(_column_iterators[_schema->unique_id(cid)]->get_row_ranges_by_zone_map( - &and_predicate, nullptr, &column_rp_row_ranges)); + RETURN_IF_ERROR( + _column_iterators[runtime_predicate->column_id()]->get_row_ranges_by_zone_map( + &and_predicate, nullptr, &column_rp_row_ranges)); // intersect different columns's row ranges to get final row ranges by zone map RowRanges::ranges_intersection(zone_map_row_ranges, column_rp_row_ranges, @@ -499,8 +502,7 @@ Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row for (auto cid : cids) { RowRanges tmp_row_ranges = RowRanges::create_single(num_rows()); DCHECK(_opts.col_id_to_predicates.count(cid) > 0); - uint32_t unique_cid = _schema->unique_id(cid); - RETURN_IF_ERROR(_column_iterators[unique_cid]->get_row_ranges_by_dict( + RETURN_IF_ERROR(_column_iterators[cid]->get_row_ranges_by_dict( _opts.col_id_to_predicates.at(cid).get(), &tmp_row_ranges)); RowRanges::ranges_intersection(dict_row_ranges, tmp_row_ranges, &dict_row_ranges); } @@ -523,19 +525,18 @@ Status SegmentIterator::_apply_bitmap_index() { std::vector remaining_predicates; for (auto pred : _col_predicates) { - int32_t unique_id = _schema->unique_id(pred->column_id()); - if (_bitmap_index_iterators.count(unique_id) < 1 || - _bitmap_index_iterators[unique_id] == nullptr || pred->type() == PredicateType::BF) { + auto cid = pred->column_id(); + if (_bitmap_index_iterators[cid] == nullptr || pred->type() == PredicateType::BF) { // no bitmap index for this column remaining_predicates.push_back(pred); } else { - RETURN_IF_ERROR(pred->evaluate(_bitmap_index_iterators[unique_id].get(), - _segment->num_rows(), &_row_bitmap)); + RETURN_IF_ERROR(pred->evaluate(_bitmap_index_iterators[cid].get(), _segment->num_rows(), + &_row_bitmap)); auto column_name = _schema->column(pred->column_id())->name(); if (_check_column_pred_all_push_down(column_name) && !pred->predicate_params()->marked_by_runtime_filter) { - _need_read_data_indices[unique_id] = false; + _need_read_data_indices[cid] = false; } if (_row_bitmap.isEmpty()) { @@ -657,9 +658,7 @@ bool SegmentIterator::_can_filter_by_preds_except_leafnode_of_andnode() { } bool SegmentIterator::_check_apply_by_bitmap_index(ColumnPredicate* pred) { - int32_t unique_id = _schema->unique_id(pred->column_id()); - if (_bitmap_index_iterators.count(unique_id) < 1 || - _bitmap_index_iterators[unique_id] == nullptr) { + if (_bitmap_index_iterators[pred->column_id()] == nullptr) { // no bitmap index for this column return false; } @@ -667,9 +666,7 @@ bool SegmentIterator::_check_apply_by_bitmap_index(ColumnPredicate* pred) { } bool SegmentIterator::_check_apply_by_inverted_index(ColumnPredicate* pred, bool pred_in_compound) { - int32_t unique_id = _schema->unique_id(pred->column_id()); - if (_inverted_index_iterators.count(unique_id) < 1 || - _inverted_index_iterators[unique_id] == nullptr) { + if (_inverted_index_iterators[pred->column_id()] == nullptr) { //this column without inverted index return false; } @@ -685,11 +682,12 @@ bool SegmentIterator::_check_apply_by_inverted_index(ColumnPredicate* pred, bool } // Function filter no apply inverted index - if (dynamic_cast(pred)) { + if (dynamic_cast*>(pred) != nullptr || + dynamic_cast*>(pred) != nullptr) { return false; } - bool handle_by_fulltext = _column_has_fulltext_index(unique_id); + bool handle_by_fulltext = _column_has_fulltext_index(pred->column_id()); if (handle_by_fulltext) { // when predicate in compound condition which except leafNode of andNode, // only can apply match query for fulltext index, @@ -705,9 +703,8 @@ bool SegmentIterator::_check_apply_by_inverted_index(ColumnPredicate* pred, bool Status SegmentIterator::_apply_bitmap_index_except_leafnode_of_andnode( ColumnPredicate* pred, roaring::Roaring* output_result) { - int32_t unique_id = _schema->unique_id(pred->column_id()); - RETURN_IF_ERROR(pred->evaluate(_bitmap_index_iterators[unique_id].get(), _segment->num_rows(), - output_result)); + RETURN_IF_ERROR(pred->evaluate(_bitmap_index_iterators[pred->column_id()].get(), + _segment->num_rows(), output_result)); return Status::OK(); } @@ -716,9 +713,8 @@ Status SegmentIterator::_apply_inverted_index_except_leafnode_of_andnode( if (_opts.runtime_state && !_opts.runtime_state->query_options().enable_inverted_index_query) { return Status::OK(); } - int32_t unique_id = _schema->unique_id(pred->column_id()); - RETURN_IF_ERROR(pred->evaluate(*_schema, _inverted_index_iterators[unique_id].get(), num_rows(), - output_result)); + RETURN_IF_ERROR(pred->evaluate(*_schema, _inverted_index_iterators[pred->column_id()].get(), + num_rows(), output_result)); return Status::OK(); } @@ -745,8 +741,7 @@ Status SegmentIterator::_apply_index_except_leafnode_of_andnode() { continue; } - int32_t unique_id = _schema->unique_id(pred->column_id()); - bool need_remaining_after_evaluate = _column_has_fulltext_index(unique_id) && + bool need_remaining_after_evaluate = _column_has_fulltext_index(pred->column_id()) && PredicateTypeTraits::is_equal_or_list(pred_type); if (!res.ok()) { if (_downgrade_without_index(res, need_remaining_after_evaluate)) { @@ -771,8 +766,7 @@ Status SegmentIterator::_apply_index_except_leafnode_of_andnode() { _check_column_pred_all_push_down(column_name, true, pred->type() == PredicateType::MATCH) && !pred->predicate_params()->marked_by_runtime_filter) { - int32_t unique_id = _schema->unique_id(pred->column_id()); - _need_read_data_indices[unique_id] = false; + _need_read_data_indices[pred->column_id()] = false; } } @@ -823,12 +817,10 @@ std::string SegmentIterator::_gen_predicate_result_sign(ColumnPredicateInfo* pre return pred_result_sign; } -bool SegmentIterator::_column_has_fulltext_index(int32_t unique_id) { - bool has_fulltext_index = - _inverted_index_iterators.count(unique_id) > 0 && - _inverted_index_iterators[unique_id] != nullptr && - _inverted_index_iterators[unique_id]->get_inverted_index_reader_type() == - InvertedIndexReaderType::FULLTEXT; +bool SegmentIterator::_column_has_fulltext_index(int32_t cid) { + bool has_fulltext_index = _inverted_index_iterators[cid] != nullptr && + _inverted_index_iterators[cid]->get_inverted_index_reader_type() == + InvertedIndexReaderType::FULLTEXT; return has_fulltext_index; } @@ -852,11 +844,10 @@ Status SegmentIterator::_apply_inverted_index_on_column_predicate( if (!_check_apply_by_inverted_index(pred)) { remaining_predicates.emplace_back(pred); } else { - int32_t unique_id = _schema->unique_id(pred->column_id()); - bool need_remaining_after_evaluate = _column_has_fulltext_index(unique_id) && + bool need_remaining_after_evaluate = _column_has_fulltext_index(pred->column_id()) && PredicateTypeTraits::is_equal_or_list(pred->type()); roaring::Roaring bitmap = _row_bitmap; - Status res = pred->evaluate(*_schema, _inverted_index_iterators[unique_id].get(), + Status res = pred->evaluate(*_schema, _inverted_index_iterators[pred->column_id()].get(), num_rows(), &bitmap); if (!res.ok()) { if (_downgrade_without_index(res, need_remaining_after_evaluate)) { @@ -891,7 +882,7 @@ Status SegmentIterator::_apply_inverted_index_on_column_predicate( if (_check_column_pred_all_push_down(column_name, false, pred->type() == PredicateType::MATCH) && !pred->predicate_params()->marked_by_runtime_filter) { - _need_read_data_indices[unique_id] = false; + _need_read_data_indices[pred->column_id()] = false; } } return Status::OK(); @@ -901,8 +892,7 @@ Status SegmentIterator::_apply_inverted_index_on_block_column_predicate( ColumnId column_id, MutilColumnBlockPredicate* pred, std::set& no_need_to_pass_column_predicate_set, bool* continue_apply) { - auto unique_id = _schema->unique_id(column_id); - bool handle_by_fulltext = _column_has_fulltext_index(unique_id); + bool handle_by_fulltext = _column_has_fulltext_index(column_id); std::set predicate_set {}; pred->get_all_column_predicate(predicate_set); @@ -912,20 +902,19 @@ Status SegmentIterator::_apply_inverted_index_on_block_column_predicate( //2. There are multiple predicates for this column. //3. All the predicates are range predicate. //4. if it's under fulltext parser type, we need to skip inverted index evaluate. - if (_inverted_index_iterators.count(unique_id) > 0 && - _inverted_index_iterators[unique_id] != nullptr && predicate_set.size() > 1 && + if (_inverted_index_iterators[column_id] != nullptr && predicate_set.size() > 1 && all_predicates_are_range_predicate(predicate_set) && !handle_by_fulltext) { roaring::Roaring output_result = _row_bitmap; std::string column_name = _schema->column(column_id)->name(); - auto res = pred->evaluate(column_name, _inverted_index_iterators[unique_id].get(), + auto res = pred->evaluate(column_name, _inverted_index_iterators[column_id].get(), num_rows(), &output_result); if (res.ok()) { if (_check_column_pred_all_push_down(column_name) && !all_predicates_are_marked_by_runtime_filter(predicate_set)) { - _need_read_data_indices[unique_id] = false; + _need_read_data_indices[column_id] = false; } no_need_to_pass_column_predicate_set.insert(predicate_set.begin(), predicate_set.end()); _row_bitmap &= output_result; @@ -957,7 +946,7 @@ bool SegmentIterator::_need_read_data(ColumnId cid) { return true; } int32_t unique_id = _opts.tablet_schema->column(cid).unique_id(); - if (_need_read_data_indices.count(unique_id) > 0 && !_need_read_data_indices[unique_id] && + if (_need_read_data_indices.count(cid) > 0 && !_need_read_data_indices[cid] && _output_columns.count(unique_id) < 1) { VLOG_DEBUG << "SegmentIterator no need read data for column: " << _opts.tablet_schema->column_by_uid(unique_id).name(); @@ -1010,7 +999,7 @@ Status SegmentIterator::_init_return_column_iterators() { for (auto cid : _schema->column_ids()) { if (_schema->column(cid)->name() == BeConsts::ROWID_COL) { - _column_iterators[_schema->column(cid)->unique_id()].reset( + _column_iterators[cid].reset( new RowIdColumnIterator(_opts.tablet_id, _opts.rowset_id, _segment->id())); continue; } @@ -1027,10 +1016,9 @@ Status SegmentIterator::_init_return_column_iterators() { tmp_is_pred_column[cid] = true; } - int32_t unique_id = _opts.tablet_schema->column(cid).unique_id(); - if (_column_iterators.count(unique_id) < 1) { + if (_column_iterators[cid] == nullptr) { RETURN_IF_ERROR(_segment->new_column_iterator(_opts.tablet_schema->column(cid), - &_column_iterators[unique_id])); + &_column_iterators[cid])); ColumnIteratorOptions iter_opts; iter_opts.stats = _opts.stats; iter_opts.use_page_cache = _opts.use_page_cache; @@ -1039,7 +1027,7 @@ Status SegmentIterator::_init_return_column_iterators() { // If the col is predicate column, then should read the last page to check // if the column is full dict encoding iter_opts.is_predicate_column = tmp_is_pred_column[cid]; - RETURN_IF_ERROR(_column_iterators[unique_id]->init(iter_opts)); + RETURN_IF_ERROR(_column_iterators[cid]->init(iter_opts)); } } return Status::OK(); @@ -1050,10 +1038,9 @@ Status SegmentIterator::_init_bitmap_index_iterators() { return Status::OK(); } for (auto cid : _schema->column_ids()) { - int32_t unique_id = _opts.tablet_schema->column(cid).unique_id(); - if (_bitmap_index_iterators.count(unique_id) < 1) { - RETURN_IF_ERROR(_segment->new_bitmap_index_iterator( - _opts.tablet_schema->column(cid), &_bitmap_index_iterators[unique_id])); + if (_bitmap_index_iterators[cid] == nullptr) { + RETURN_IF_ERROR(_segment->new_bitmap_index_iterator(_opts.tablet_schema->column(cid), + &_bitmap_index_iterators[cid])); } } return Status::OK(); @@ -1064,11 +1051,10 @@ Status SegmentIterator::_init_inverted_index_iterators() { return Status::OK(); } for (auto cid : _schema->column_ids()) { - int32_t unique_id = _opts.tablet_schema->column(cid).unique_id(); - if (_inverted_index_iterators.count(unique_id) < 1) { + if (_inverted_index_iterators[cid] == nullptr) { RETURN_IF_ERROR(_segment->new_inverted_index_iterator( _opts.tablet_schema->column(cid), _opts.tablet_schema->get_inverted_index(cid), - _opts.stats, &_inverted_index_iterators[unique_id])); + _opts.stats, &_inverted_index_iterators[cid])); } } return Status::OK(); @@ -1241,7 +1227,7 @@ Status SegmentIterator::_seek_columns(const std::vector& column_ids, r if (!_need_read_data(cid)) { continue; } - RETURN_IF_ERROR(_column_iterators[_schema->unique_id(cid)]->seek_to_ordinal(pos)); + RETURN_IF_ERROR(_column_iterators[cid]->seek_to_ordinal(pos)); } return Status::OK(); } @@ -1328,7 +1314,9 @@ Status SegmentIterator::_vec_init_lazy_materialization() { } else { short_cir_pred_col_id_set.insert(cid); _short_cir_eval_predicate.push_back(predicate); - _filter_info_id.push_back(predicate); + if (predicate->is_filter()) { + _filter_info_id.push_back(predicate); + } } } @@ -1460,7 +1448,7 @@ bool SegmentIterator::_can_evaluated_by_vectorized(ColumnPredicate* predicate) { field_type == FieldType::OLAP_FIELD_TYPE_STRING) { return config::enable_low_cardinality_optimize && _opts.io_ctx.reader_type == ReaderType::READER_QUERY && - _column_iterators[_schema->unique_id(cid)]->is_all_dict_encoding(); + _column_iterators[cid]->is_all_dict_encoding(); } else if (field_type == FieldType::OLAP_FIELD_TYPE_DECIMAL) { return false; } @@ -1519,7 +1507,7 @@ Status SegmentIterator::_read_columns(const std::vector& column_ids, if (_prune_column(cid, column, true, rows_read)) { continue; } - RETURN_IF_ERROR(_column_iterators[_schema->unique_id(cid)]->next_batch(&rows_read, column)); + RETURN_IF_ERROR(_column_iterators[cid]->next_batch(&rows_read, column)); if (nrows != rows_read) { return Status::Error("nrows({}) != rows_read({})", nrows, rows_read); @@ -1733,8 +1721,8 @@ Status SegmentIterator::_read_columns_by_rowids(std::vector& read_colu if (_prune_column(cid, (*mutable_columns)[cid], true, select_size)) { continue; } - RETURN_IF_ERROR(_column_iterators[_schema->unique_id(cid)]->read_by_rowids( - rowids.data(), select_size, _current_return_columns[cid])); + RETURN_IF_ERROR(_column_iterators[cid]->read_by_rowids(rowids.data(), select_size, + _current_return_columns[cid])); } return Status::OK(); diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index a25cf1b5bffa4ca..a44d0b6ebaee4c4 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -187,7 +187,7 @@ class SegmentIterator : public RowwiseIterator { ColumnPredicate* pred, roaring::Roaring* output_result); [[nodiscard]] Status _apply_inverted_index_except_leafnode_of_andnode( ColumnPredicate* pred, roaring::Roaring* output_result); - bool _column_has_fulltext_index(int32_t unique_id); + bool _column_has_fulltext_index(int32_t cid); bool _downgrade_without_index(Status res, bool need_remaining = false); inline bool _inverted_index_not_support_pred_type(const PredicateType& type); bool _can_filter_by_preds_except_leafnode_of_andnode(); @@ -331,12 +331,10 @@ class SegmentIterator : public RowwiseIterator { std::shared_ptr _segment; SchemaSPtr _schema; - // _column_iterators_map.size() == _schema.num_columns() - // map _column_iterators_map/_bitmap_index_iterators; - // can use _schema get unique_id by cid - std::map> _column_iterators; - std::map> _bitmap_index_iterators; - std::map> _inverted_index_iterators; + // vector idx -> column iterarator + std::vector> _column_iterators; + std::vector> _bitmap_index_iterators; + std::vector> _inverted_index_iterators; // after init(), `_row_bitmap` contains all rowid to scan roaring::Roaring _row_bitmap; // "column_name+operator+value-> diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp index bd3e31596a6b1cc..639e2ec08110e4d 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -419,7 +419,7 @@ Status SegmentWriter::append_block_with_partial_content(const vectorized::Block* RowsetSharedPtr rowset; auto st = _tablet->lookup_row_key(key, have_input_seq_column, specified_rowsets, &loc, _mow_context->max_version, segment_caches, &rowset); - if (st.is()) { + if (st.is()) { if (_tablet_schema->is_strict_mode()) { ++num_rows_filtered; // delete the invalid newly inserted row @@ -436,7 +436,7 @@ Status SegmentWriter::append_block_with_partial_content(const vectorized::Block* use_default_or_null_flag.emplace_back(true); continue; } - if (!st.ok() && !st.is()) { + if (!st.ok() && !st.is()) { LOG(WARNING) << "failed to lookup row key, error: " << st; return st; } @@ -454,7 +454,7 @@ Status SegmentWriter::append_block_with_partial_content(const vectorized::Block* _tablet->prepare_to_read(loc, segment_pos, &_rssid_to_rid); } - if (st.is()) { + if (st.is()) { // although we need to mark delete current row, we still need to read missing columns // for this row, we need to ensure that each column is aligned _mow_context->delete_bitmap->add({_opts.rowset_ctx->rowset_id, _segment_id, 0}, diff --git a/be/src/olap/rowset/segment_v2/zone_map_index.cpp b/be/src/olap/rowset/segment_v2/zone_map_index.cpp index 9a4ec628c2f266a..4306224c13162c7 100644 --- a/be/src/olap/rowset/segment_v2/zone_map_index.cpp +++ b/be/src/olap/rowset/segment_v2/zone_map_index.cpp @@ -145,8 +145,17 @@ Status TypedZoneMapIndexWriter::finish(io::FileWriter* file_writer, return writer.finish(meta->mutable_page_zone_maps()); } -Status ZoneMapIndexReader::load(bool use_page_cache, bool kept_in_memory) { - IndexedColumnReader reader(_file_reader, _index_meta->page_zone_maps()); +Status ZoneMapIndexReader::load(bool use_page_cache, bool kept_in_memory, + const ZoneMapIndexPB* index_meta) { + // TODO yyq: implement a new once flag to avoid status construct. + return _load_once.call([this, use_page_cache, kept_in_memory, index_meta] { + return _load(use_page_cache, kept_in_memory, index_meta); + }); +} + +Status ZoneMapIndexReader::_load(bool use_page_cache, bool kept_in_memory, + const ZoneMapIndexPB* index_meta) { + IndexedColumnReader reader(_file_reader, index_meta->page_zone_maps()); RETURN_IF_ERROR(reader.load(use_page_cache, kept_in_memory)); IndexedColumnIterator iter(&reader); diff --git a/be/src/olap/rowset/segment_v2/zone_map_index.h b/be/src/olap/rowset/segment_v2/zone_map_index.h index ed3418dc48e661d..b6a6ae90751d249 100644 --- a/be/src/olap/rowset/segment_v2/zone_map_index.h +++ b/be/src/olap/rowset/segment_v2/zone_map_index.h @@ -30,6 +30,7 @@ #include "io/fs/file_reader_writer_fwd.h" #include "olap/field.h" #include "runtime/define_primitive_type.h" +#include "util/once.h" #include "vec/common/arena.h" namespace doris { @@ -146,20 +147,23 @@ class TypedZoneMapIndexWriter final : public ZoneMapIndexWriter { class ZoneMapIndexReader { public: - explicit ZoneMapIndexReader(io::FileReaderSPtr file_reader, const ZoneMapIndexPB* index_meta) - : _file_reader(std::move(file_reader)), _index_meta(index_meta) {} + explicit ZoneMapIndexReader(io::FileReaderSPtr file_reader) + : _file_reader(std::move(file_reader)) {} // load all page zone maps into memory - Status load(bool use_page_cache, bool kept_in_memory); + Status load(bool use_page_cache, bool kept_in_memory, const ZoneMapIndexPB*); const std::vector& page_zone_maps() const { return _page_zone_maps; } int32_t num_pages() const { return _page_zone_maps.size(); } private: - io::FileReaderSPtr _file_reader; - const ZoneMapIndexPB* _index_meta; + Status _load(bool use_page_cache, bool kept_in_memory, const ZoneMapIndexPB*); +private: + DorisCallOnce _load_once; + // TODO: yyq, we shoud remove file_reader from here. + io::FileReaderSPtr _file_reader; std::vector _page_zone_maps; }; diff --git a/be/src/olap/rowset_builder.cpp b/be/src/olap/rowset_builder.cpp index f92b1c96fb1a784..aad17ce2fdfdc70 100644 --- a/be/src/olap/rowset_builder.cpp +++ b/be/src/olap/rowset_builder.cpp @@ -261,8 +261,8 @@ Status RowsetBuilder::commit_txn() { } if (_tablet->enable_unique_key_merge_on_write()) { _storage_engine->txn_manager()->set_txn_related_delete_bitmap( - _req.partition_id, _req.txn_id, _tablet->tablet_id(), _tablet->schema_hash(), - _tablet->tablet_uid(), true, _delete_bitmap, _rowset_ids); + _req.partition_id, _req.txn_id, _tablet->tablet_id(), _tablet->tablet_uid(), true, + _delete_bitmap, _rowset_ids); } _is_committed = true; diff --git a/be/src/olap/schema.cpp b/be/src/olap/schema.cpp index 95f6a47a701110d..d465db4f1d0fb47 100644 --- a/be/src/olap/schema.cpp +++ b/be/src/olap/schema.cpp @@ -171,6 +171,13 @@ vectorized::IColumn::MutablePtr Schema::get_predicate_column_ptr(const Field& fi ptr = doris::vectorized::PredicateColumnType::create(); break; case FieldType::OLAP_FIELD_TYPE_CHAR: + if (config::enable_low_cardinality_optimize && reader_type == ReaderType::READER_QUERY) { + ptr = doris::vectorized::ColumnDictionary::create( + field.type()); + } else { + ptr = doris::vectorized::PredicateColumnType::create(); + } + break; case FieldType::OLAP_FIELD_TYPE_VARCHAR: case FieldType::OLAP_FIELD_TYPE_STRING: if (config::enable_low_cardinality_optimize && reader_type == ReaderType::READER_QUERY) { diff --git a/be/src/olap/schema_cache.h b/be/src/olap/schema_cache.h index f34f7c296d81689..5d94c928371117a 100644 --- a/be/src/olap/schema_cache.h +++ b/be/src/olap/schema_cache.h @@ -117,7 +117,7 @@ class SchemaCache : public LRUCachePolicy { private: SchemaCache(size_t capacity) - : LRUCachePolicy("SchemaCache", capacity, LRUCacheType::NUMBER, + : LRUCachePolicy(CachePolicy::CacheType::SCHEMA_CACHE, capacity, LRUCacheType::NUMBER, config::schema_cache_sweep_time_sec) {} static constexpr char SCHEMA_DELIMITER = '-'; static SchemaCache* _s_instance; diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 7dea2906067028c..48cb9367ae01c9f 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -283,11 +283,8 @@ Status BlockChanger::change_block(vectorized::Block* ref_block, const int column_size = new_block->columns(); // swap ref_block[key] and new_block[value] - std::map swap_idx_map; - + std::list> swap_idx_list; for (int idx = 0; idx < column_size; idx++) { - int ref_idx = _schema_mapping[idx].ref_column; - if (_schema_mapping[idx].expr != nullptr) { vectorized::VExprContextSPtr ctx; RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(*_schema_mapping[idx].expr, ctx)); @@ -303,14 +300,11 @@ Status BlockChanger::change_block(vectorized::Block* ref_block, "{} size invalid, expect={}, real={}", new_block->get_by_position(idx).name, row_size, ref_block->get_by_position(result_column_id).column->size()); } - - if (_type != ROLLUP) { - RETURN_IF_ERROR( - _check_cast_valid(ref_block->get_by_position(ref_idx).column, - ref_block->get_by_position(result_column_id).column)); - } - swap_idx_map[result_column_id] = idx; - } else if (ref_idx < 0) { + RETURN_IF_ERROR(_check_cast_valid(ref_block->get_by_position(idx).column, + ref_block->get_by_position(result_column_id).column, + _type)); + swap_idx_list.push_back({result_column_id, idx}); + } else if (_schema_mapping[idx].ref_column < 0) { if (_type != ROLLUP) { // new column, write default value auto value = _schema_mapping[idx].default_value; @@ -330,24 +324,24 @@ Status BlockChanger::change_block(vectorized::Block* ref_block, } } else { // same type, just swap column - swap_idx_map[ref_idx] = idx; + swap_idx_list.push_back({_schema_mapping[idx].ref_column, idx}); } } - for (auto it : swap_idx_map) { - auto& ref_col = ref_block->get_by_position(it.first); - auto& new_col = new_block->get_by_position(it.second); + for (auto it : swap_idx_list) { + auto& ref_col = ref_block->get_by_position(it.first).column; + auto& new_col = new_block->get_by_position(it.second).column; - bool ref_col_nullable = ref_col.column->is_nullable(); - bool new_col_nullable = new_col.column->is_nullable(); + bool ref_col_nullable = ref_col->is_nullable(); + bool new_col_nullable = new_col->is_nullable(); if (ref_col_nullable != new_col_nullable) { // not nullable to nullable if (new_col_nullable) { - auto* new_nullable_col = assert_cast( - new_col.column->assume_mutable().get()); + auto* new_nullable_col = + assert_cast(new_col->assume_mutable().get()); - new_nullable_col->swap_nested_column(ref_col.column); + new_nullable_col->change_nested_column(ref_col); new_nullable_col->get_null_map_data().resize_fill(new_nullable_col->size()); } else { // nullable to not nullable: @@ -355,14 +349,14 @@ Status BlockChanger::change_block(vectorized::Block* ref_block, // then do schema change `alter table test modify column c_phone int not null`, // the cast expr of schema change is `CastExpr(CAST String to Nullable(Int32))`, // so need to handle nullable to not nullable here - auto* ref_nullable_col = assert_cast( - ref_col.column->assume_mutable().get()); + auto* ref_nullable_col = + assert_cast(ref_col->assume_mutable().get()); - ref_nullable_col->swap_nested_column(new_col.column); + new_col = ref_nullable_col->get_nested_column_ptr(); } } else { - new_block->get_by_position(it.second).column.swap( - ref_block->get_by_position(it.first).column); + new_block->get_by_position(it.second).column = + ref_block->get_by_position(it.first).column; } } return Status::OK(); @@ -370,7 +364,16 @@ Status BlockChanger::change_block(vectorized::Block* ref_block, // This check is to prevent schema-change from causing data loss Status BlockChanger::_check_cast_valid(vectorized::ColumnPtr ref_column, - vectorized::ColumnPtr new_column) const { + vectorized::ColumnPtr new_column, + AlterTabletType type) const { + if (ref_column->size() != new_column->size()) { + return Status::InternalError( + "column size is changed, ref_column_size={}, new_column_size={}", + ref_column->size(), new_column->size()); + } + if (type == ROLLUP) { + return Status::OK(); + } if (ref_column->is_nullable() != new_column->is_nullable()) { if (ref_column->is_nullable()) { auto* ref_null_map = @@ -487,10 +490,7 @@ Status VSchemaChangeDirectly::_inner_process(RowsetReaderSharedPtr rowset_reader RETURN_IF_ERROR(rowset_writer->add_block(new_block.get())); } while (true); - if (!rowset_writer->flush()) { - return Status::Error("rowset_writer flush failed"); - } - + RETURN_IF_ERROR(rowset_writer->flush()); return Status::OK(); } @@ -593,7 +593,6 @@ Status VSchemaChangeWithSorting::_internal_sorting( SegmentsOverlapPB segments_overlap, RowsetSharedPtr* rowset) { uint64_t merged_rows = 0; MultiBlockMerger merger(new_tablet); - std::unique_ptr rowset_writer; RowsetWriterContext context; context.version = version; @@ -630,7 +629,6 @@ Status VSchemaChangeWithSorting::_external_sorting(vector& src_ RETURN_IF_ERROR(Merger::vmerge_rowsets(new_tablet, ReaderType::READER_ALTER_TABLE, new_tablet->tablet_schema(), rs_readers, rowset_writer, &stats)); - _add_merged_rows(stats.merged_rows); _add_filtered_rows(stats.filtered_rows); return Status::OK(); diff --git a/be/src/olap/schema_change.h b/be/src/olap/schema_change.h index f9db4c488aa31f5..2c5075b9ce5845b 100644 --- a/be/src/olap/schema_change.h +++ b/be/src/olap/schema_change.h @@ -83,8 +83,8 @@ class BlockChanger { bool has_where() const { return _where_expr != nullptr; } private: - Status _check_cast_valid(vectorized::ColumnPtr ref_column, - vectorized::ColumnPtr new_column) const; + Status _check_cast_valid(vectorized::ColumnPtr ref_column, vectorized::ColumnPtr new_column, + AlterTabletType type) const; // @brief column-mapping specification of new schema SchemaMapping _schema_mapping; diff --git a/be/src/olap/segment_loader.cpp b/be/src/olap/segment_loader.cpp index 4704f8e802d0b8a..a0350dcfc2f1cc8 100644 --- a/be/src/olap/segment_loader.cpp +++ b/be/src/olap/segment_loader.cpp @@ -70,14 +70,14 @@ Status SegmentLoader::load_segments(const BetaRowsetSharedPtr& rowset, } SegmentCache::CacheKey cache_key(rowset->rowset_id()); - if (_segment_cache->lookup(cache_key, cache_handle)) { + if (!config::disable_segment_cache && _segment_cache->lookup(cache_key, cache_handle)) { return Status::OK(); } std::vector segments; RETURN_IF_ERROR(rowset->load_segments(&segments)); - if (use_cache) { + if (use_cache && !config::disable_segment_cache) { // memory of SegmentCache::CacheValue will be handled by SegmentCache SegmentCache::CacheValue* cache_value = new SegmentCache::CacheValue(); cache_value->segments = std::move(segments); diff --git a/be/src/olap/segment_loader.h b/be/src/olap/segment_loader.h index 589a930201c57c1..ce0d00909a5cb66 100644 --- a/be/src/olap/segment_loader.h +++ b/be/src/olap/segment_loader.h @@ -73,7 +73,7 @@ class SegmentCache : public LRUCachePolicy { }; SegmentCache(size_t capacity) - : LRUCachePolicy("SegmentCache", capacity, LRUCacheType::NUMBER, + : LRUCachePolicy(CachePolicy::CacheType::SEGMENT_CACHE, capacity, LRUCacheType::NUMBER, config::tablet_rowset_stale_sweep_time_sec) {} // Lookup the given rowset in the cache. diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index 7956d41cd29e925..b25e0862fc216df 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -103,13 +103,14 @@ static Status _validate_options(const EngineOptions& options) { return Status::OK(); } -Status StorageEngine::open(const EngineOptions& options, StorageEngine** engine_ptr) { +Status StorageEngine::open(const EngineOptions& options, + std::unique_ptr* engine_ptr) { RETURN_IF_ERROR(_validate_options(options)); LOG(INFO) << "starting backend using uid:" << options.backend_uid.to_string(); std::unique_ptr engine(new StorageEngine(options)); RETURN_NOT_OK_STATUS_WITH_WARN(engine->_open(), "open engine failed"); - *engine_ptr = engine.release(); LOG(INFO) << "success to init storage engine."; + *engine_ptr = std::move(engine); return Status::OK(); } @@ -138,6 +139,8 @@ StorageEngine::StorageEngine(const EngineOptions& options) } StorageEngine::~StorageEngine() { + stop(); + DEREGISTER_HOOK_METRIC(unused_rowsets_count); if (_base_compaction_thread_pool) { @@ -163,21 +166,29 @@ StorageEngine::~StorageEngine() { _s_instance = nullptr; } -void StorageEngine::load_data_dirs(const std::vector& data_dirs) { +Status StorageEngine::load_data_dirs(const std::vector& data_dirs) { std::vector threads; - for (auto data_dir : data_dirs) { - threads.emplace_back([data_dir] { - auto res = data_dir->load(); - if (!res.ok()) { - LOG(WARNING) << "io error when init load tables. res=" << res - << ", data dir=" << data_dir->path(); - // TODO(lingbin): why not exit progress, to force OP to change the conf - } - }); + std::vector results(data_dirs.size()); + for (size_t i = 0; i < data_dirs.size(); ++i) { + threads.emplace_back( + [&results, data_dir = data_dirs[i]](size_t index) { + results[index] = data_dir->load(); + if (!results[index].ok()) { + LOG(WARNING) << "io error when init load tables. res=" << results[index] + << ", data dir=" << data_dir->path(); + } + }, + i); } for (auto& thread : threads) { thread.join(); } + for (const auto& result : results) { + if (!result.ok()) { + return result; + } + } + return Status::OK(); } Status StorageEngine::_open() { @@ -192,7 +203,7 @@ Status StorageEngine::_open() { RETURN_NOT_OK_STATUS_WITH_WARN(_check_file_descriptor_number(), "check fd number failed"); auto dirs = get_stores(); - load_data_dirs(dirs); + RETURN_IF_ERROR(load_data_dirs(dirs)); _memtable_flush_executor.reset(new MemTableFlushExecutor()); _memtable_flush_executor->init(dirs); @@ -533,6 +544,7 @@ void StorageEngine::_exit_if_too_many_disks_are_failed() { } void StorageEngine::stop() { + if (_stopped) return; // trigger the waiting threads notify_listeners(); @@ -882,7 +894,6 @@ void StorageEngine::_clean_unused_txns() { // currently just remove them from memory // nullptr to indicate not remove them from meta store _txn_manager->force_rollback_tablet_related_txns(nullptr, tablet_info.tablet_id, - tablet_info.schema_hash, tablet_info.tablet_uid); } } diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index 9ab23463f25ae59..822f0a458bfffd2 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -81,7 +81,7 @@ class StorageEngine { StorageEngine(const EngineOptions& options); ~StorageEngine(); - static Status open(const EngineOptions& options, StorageEngine** engine_ptr); + static Status open(const EngineOptions& options, std::unique_ptr* engine_ptr); static StorageEngine* instance() { return _s_instance; } @@ -91,14 +91,14 @@ class StorageEngine { void clear_transaction_task(const TTransactionId transaction_id, const std::vector& partition_ids); - // Note: 这里只能reload原先已经存在的root path,即re-load启动时就登记的root path - // 是允许的,但re-load全新的path是不允许的,因为此处没有彻底更新ce调度器信息 - void load_data_dirs(const std::vector& stores); + // Note: Only the previously existing root path can be reloaded here, that is, the root path registered when re load starts is allowed, + // but the brand new path of re load is not allowed because the ce scheduler information has not been thoroughly updated here + Status load_data_dirs(const std::vector& stores); template std::vector get_stores(); - // @brief 获取所有root_path信息 + // get all info of root_path Status get_all_data_dir_info(std::vector* data_dir_infos, bool need_update); int64_t get_file_or_directory_size(const std::string& file_path); @@ -483,6 +483,8 @@ class StorageEngine { scoped_refptr _async_publish_thread; std::mutex _async_publish_mutex; + bool _clear_segment_cache = false; + DISALLOW_COPY_AND_ASSIGN(StorageEngine); }; diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index e1921327343aa68..afae3fa7e2b26d1 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -676,6 +676,9 @@ void Tablet::delete_expired_stale_rowset() { // Compute the end time to delete rowsets, when a expired rowset createtime less then this time, it will be deleted. double expired_stale_sweep_endtime = ::difftime(now, config::tablet_rowset_stale_sweep_time_sec); + if (config::tablet_rowset_stale_sweep_by_size) { + expired_stale_sweep_endtime = now; + } std::vector path_id_vec; // capture the path version to delete @@ -1270,7 +1273,7 @@ Status Tablet::_contains_version(const Version& version) { } TabletInfo Tablet::get_tablet_info() const { - return TabletInfo(tablet_id(), schema_hash(), tablet_uid()); + return TabletInfo(tablet_id(), tablet_uid()); } std::vector Tablet::pick_candidate_rowsets_to_cumulative_compaction() { @@ -2830,10 +2833,10 @@ Status Tablet::lookup_row_key(const Slice& encoded_key, bool with_seq_col, for (auto id : picked_segments) { Status s = segments[id]->lookup_row_key(encoded_key, with_seq_col, &loc); - if (s.is()) { + if (s.is()) { continue; } - if (!s.ok() && !s.is()) { + if (!s.ok() && !s.is()) { return s; } if (s.ok() && _tablet_meta->delete_bitmap().contains_agg_without_cache( @@ -2846,7 +2849,7 @@ Status Tablet::lookup_row_key(const Slice& encoded_key, bool with_seq_col, // The key is deleted, we don't need to search for it any more. break; } - // `st` is either OK or ALREADY_EXIST now. + // `st` is either OK or KEY_ALREADY_EXISTS now. // for partial update, even if the key is already exists, we still need to // read it's original values to keep all columns align. *row_location = loc; @@ -2859,7 +2862,7 @@ Status Tablet::lookup_row_key(const Slice& encoded_key, bool with_seq_col, } } g_tablet_pk_not_found << 1; - return Status::NotFound("can't find key in all rowsets"); + return Status::Error("can't find key in all rowsets"); } // load segment may do io so it should out lock @@ -2973,17 +2976,17 @@ Status Tablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset, RowsetSharedPtr rowset_find; auto st = lookup_row_key(key, true, specified_rowsets, &loc, dummy_version.first - 1, segment_caches, &rowset_find); - bool expected_st = st.ok() || st.is() || st.is(); + bool expected_st = st.ok() || st.is() || st.is(); DCHECK(expected_st) << "unexpected error status while lookup_row_key:" << st; if (!expected_st) { return st; } - if (st.is()) { + if (st.is()) { continue; } // sequence id smaller than the previous one, so delete current row - if (st.is()) { + if (st.is()) { delete_bitmap->add({rowset_id, seg->id(), 0}, row_id); continue; } else if (is_partial_update && rowset_writer != nullptr) { @@ -3213,26 +3216,6 @@ void Tablet::prepare_to_read(const RowLocation& row_location, size_t pos, seg_it->second.emplace_back(RidAndPos {row_location.row_id, pos}); } -Status Tablet::_check_pk_in_pre_segments( - RowsetId rowset_id, const std::vector& pre_segments, - const Slice& key, DeleteBitmapPtr delete_bitmap, RowLocation* loc) { - for (auto it = pre_segments.rbegin(); it != pre_segments.rend(); ++it) { - auto st = (*it)->lookup_row_key(key, true, loc); - DCHECK(st.ok() || st.is() || st.is()) - << "unexpected error status while lookup_row_key:" << st; - if (st.is()) { - continue; - } else if (st.ok() && _schema->has_sequence_col() && - delete_bitmap->contains({rowset_id, loc->segment_id, 0}, loc->row_id)) { - // if has sequence col, we continue to compare the sequence_id of - // all segments, util we find an existing key. - continue; - } - return st; - } - return Status::NotFound("Can't find key in the segment"); -} - void Tablet::_rowset_ids_difference(const RowsetIdUnorderedSet& cur, const RowsetIdUnorderedSet& pre, RowsetIdUnorderedSet* to_add, RowsetIdUnorderedSet* to_del) { @@ -3300,7 +3283,7 @@ Status Tablet::commit_phase_update_delete_bitmap( std::vector specified_rowsets; { std::shared_lock meta_rlock(_meta_lock); - cur_version = max_version().second; + cur_version = max_version_unlocked().second; cur_rowset_ids = all_rs_id(cur_version); _rowset_ids_difference(cur_rowset_ids, pre_rowset_ids, &rowset_ids_to_add, &rowset_ids_to_del); diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index 8ba3f60c3d7ba77..5ea64cce78ea3e8 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -580,10 +580,6 @@ class Tablet : public BaseTablet { bool _reconstruct_version_tracker_if_necessary(); void _init_context_common_fields(RowsetWriterContext& context); - Status _check_pk_in_pre_segments(RowsetId rowset_id, - const std::vector& pre_segments, - const Slice& key, DeleteBitmapPtr delete_bitmap, - RowLocation* loc); void _rowset_ids_difference(const RowsetIdUnorderedSet& cur, const RowsetIdUnorderedSet& pre, RowsetIdUnorderedSet* to_add, RowsetIdUnorderedSet* to_del); Status _load_rowset_segments(const RowsetSharedPtr& rowset, diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index d6e4294a17aeb4c..5f6020f60aead9d 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -1045,7 +1045,7 @@ Status TabletManager::build_all_report_tablets_info(std::map TTabletInfo& tablet_info = t_tablet.tablet_infos.emplace_back(); tablet->build_tablet_report_info(&tablet_info, true, true); // find expired transaction corresponding to this tablet - TabletInfo tinfo(tablet->tablet_id(), tablet->schema_hash(), tablet->tablet_uid()); + TabletInfo tinfo(tablet->tablet_id(), tablet->tablet_uid()); auto find = expire_txn_map.find(tinfo); if (find != expire_txn_map.end()) { tablet_info.__set_transaction_ids(find->second); @@ -1429,8 +1429,7 @@ void TabletManager::get_tablets_distribution_on_different_disks( DataDir* data_dir = tablet->data_dir(); size_t tablet_footprint = tablet->tablet_footprint(); tablets_num[data_dir]++; - TabletSize tablet_size(tablet_info_iter->tablet_id, tablet_info_iter->schema_hash, - tablet_footprint); + TabletSize tablet_size(tablet_info_iter->tablet_id, tablet_footprint); tablets_info[data_dir].push_back(tablet_size); } tablets_num_on_disk[partition_id] = tablets_num; diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h index c9ba12632824a17..c98e5542f7a9441 100644 --- a/be/src/olap/tablet_schema.h +++ b/be/src/olap/tablet_schema.h @@ -300,7 +300,7 @@ class TabletSchema { if (str.size() > 1) { str += ", "; } - str += p.first; + str += p.first + "(" + std::to_string(_cols[p.second].unique_id()) + ")"; } str += "]"; return str; diff --git a/be/src/olap/tablet_schema_cache.cpp b/be/src/olap/tablet_schema_cache.cpp index e14c3f7eccffcb1..2e4b221492bb05e 100644 --- a/be/src/olap/tablet_schema_cache.cpp +++ b/be/src/olap/tablet_schema_cache.cpp @@ -19,6 +19,10 @@ namespace doris { +TabletSchemaCache::~TabletSchemaCache() { + stop_and_join(); +} + TabletSchemaSPtr TabletSchemaCache::insert(const std::string& key) { DCHECK(_s_instance != nullptr); std::lock_guard guard(_mtx); diff --git a/be/src/olap/tablet_schema_cache.h b/be/src/olap/tablet_schema_cache.h index f6c1e61de39f2ec..6c692aaf46600ef 100644 --- a/be/src/olap/tablet_schema_cache.h +++ b/be/src/olap/tablet_schema_cache.h @@ -30,6 +30,8 @@ namespace doris { class TabletSchemaCache { public: + ~TabletSchemaCache(); + static void create_global_schema_cache() { DCHECK(_s_instance == nullptr); static TabletSchemaCache instance; diff --git a/be/src/olap/task/engine_publish_version_task.cpp b/be/src/olap/task/engine_publish_version_task.cpp index 3bafeca033d6c9f..1353e6691591abb 100644 --- a/be/src/olap/task/engine_publish_version_task.cpp +++ b/be/src/olap/task/engine_publish_version_task.cpp @@ -120,9 +120,8 @@ Status EnginePublishVersionTask::finish() { TabletInfo tablet_info = tablet_rs.first; RowsetSharedPtr rowset = tablet_rs.second; VLOG_CRITICAL << "begin to publish version on tablet. " - << "tablet_id=" << tablet_info.tablet_id - << ", schema_hash=" << tablet_info.schema_hash - << ", version=" << version.first << ", transaction_id=" << transaction_id; + << "tablet_id=" << tablet_info.tablet_id << ", version=" << version.first + << ", transaction_id=" << transaction_id; // if rowset is null, it means this be received write task, but failed during write // and receive fe's publish version task // this be must return as an error tablet @@ -138,8 +137,8 @@ Status EnginePublishVersionTask::finish() { if (tablet == nullptr) { _error_tablet_ids->push_back(tablet_info.tablet_id); res = Status::Error( - "can't get tablet when publish version. tablet_id={}, schema_hash={}", - tablet_info.tablet_id, tablet_info.schema_hash); + "can't get tablet when publish version. tablet_id={}", + tablet_info.tablet_id); continue; } // in uniq key model with merge-on-write, we should see all @@ -299,8 +298,7 @@ void AsyncTabletPublishTask::handle() { std::map tablet_related_rs; StorageEngine::instance()->txn_manager()->get_txn_related_tablets( _transaction_id, _partition_id, &tablet_related_rs); - auto iter = tablet_related_rs.find( - TabletInfo(_tablet->tablet_id(), _tablet->schema_hash(), _tablet->tablet_uid())); + auto iter = tablet_related_rs.find(TabletInfo(_tablet->tablet_id(), _tablet->tablet_uid())); if (iter == tablet_related_rs.end()) { return; } diff --git a/be/src/olap/task/engine_storage_migration_task.cpp b/be/src/olap/task/engine_storage_migration_task.cpp index 262123866e573a9..93c47078ce95087 100644 --- a/be/src/olap/task/engine_storage_migration_task.cpp +++ b/be/src/olap/task/engine_storage_migration_task.cpp @@ -103,8 +103,7 @@ Status EngineStorageMigrationTask::_check_running_txns() { std::set transaction_ids; // check if this tablet has related running txns. if yes, can not do migration. StorageEngine::instance()->txn_manager()->get_tablet_related_txns( - _tablet->tablet_id(), _tablet->schema_hash(), _tablet->tablet_uid(), &partition_id, - &transaction_ids); + _tablet->tablet_id(), _tablet->tablet_uid(), &partition_id, &transaction_ids); if (transaction_ids.size() > 0) { return Status::InternalError("tablet {} has unfinished txns", _tablet->tablet_id()); } diff --git a/be/src/olap/task/index_builder.cpp b/be/src/olap/task/index_builder.cpp index c621f77f498f33f..4593aecc825cad5 100644 --- a/be/src/olap/task/index_builder.cpp +++ b/be/src/olap/task/index_builder.cpp @@ -298,6 +298,33 @@ Status IndexBuilder::_add_nullable(const std::string& column_name, } return step; }; + if (field->type() == FieldType::OLAP_FIELD_TYPE_ARRAY) { + DCHECK(field->get_sub_field_count() == 1); + BitmapIterator null_iter(null_map, num_rows); + bool is_null = false; + size_t this_run = 0; + while ((this_run = null_iter.Next(&is_null)) > 0) { + if (is_null) { + RETURN_IF_ERROR(_inverted_index_builders[index_writer_sign]->add_nulls(this_run)); + } else { + // [size, offset_ptr, item_data_ptr, item_nullmap_ptr] + auto data_ptr = reinterpret_cast(*ptr); + // total number length + size_t element_cnt = size_t((unsigned long)(*data_ptr)); + auto offset_data = *(data_ptr + 1); + const uint8_t* offsets_ptr = (const uint8_t*)offset_data; + if (element_cnt > 0) { + auto data = *(data_ptr + 2); + auto nested_null_map = *(data_ptr + 3); + RETURN_IF_ERROR(_inverted_index_builders[index_writer_sign]->add_array_values( + field->get_sub_field(0)->size(), reinterpret_cast(data), + reinterpret_cast(nested_null_map), offsets_ptr, + num_rows)); + } + } + } + return Status::OK(); + } try { do { diff --git a/be/src/olap/txn_manager.cpp b/be/src/olap/txn_manager.cpp index 99eb040d09b5d91..cd29d1fd1221239 100644 --- a/be/src/olap/txn_manager.cpp +++ b/be/src/olap/txn_manager.cpp @@ -84,19 +84,17 @@ Status TxnManager::prepare_txn(TPartitionId partition_id, const TabletSharedPtr& TTransactionId transaction_id, const PUniqueId& load_id, bool ingest) { const auto& tablet_id = tablet->tablet_id(); - const auto& schema_hash = tablet->schema_hash(); const auto& tablet_uid = tablet->tablet_uid(); - return prepare_txn(partition_id, transaction_id, tablet_id, schema_hash, tablet_uid, load_id, - ingest); + return prepare_txn(partition_id, transaction_id, tablet_id, tablet_uid, load_id, ingest); } // most used for ut Status TxnManager::prepare_txn(TPartitionId partition_id, TTransactionId transaction_id, - TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, - const PUniqueId& load_id, bool ingest) { + TTabletId tablet_id, TabletUid tablet_uid, const PUniqueId& load_id, + bool ingest) { TxnKey key(partition_id, transaction_id); - TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); + TabletInfo tablet_info(tablet_id, tablet_uid); std::lock_guard txn_wrlock(_get_txn_map_lock(transaction_id)); txn_tablet_map_t& txn_tablet_map = _get_txn_tablet_map(transaction_id); @@ -154,39 +152,35 @@ Status TxnManager::commit_txn(TPartitionId partition_id, const TabletSharedPtr& TTransactionId transaction_id, const PUniqueId& load_id, const RowsetSharedPtr& rowset_ptr, bool is_recovery) { return commit_txn(tablet->data_dir()->get_meta(), partition_id, transaction_id, - tablet->tablet_id(), tablet->schema_hash(), tablet->tablet_uid(), load_id, - rowset_ptr, is_recovery); + tablet->tablet_id(), tablet->tablet_uid(), load_id, rowset_ptr, is_recovery); } Status TxnManager::publish_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, TTransactionId transaction_id, const Version& version, TabletPublishStatistics* stats) { return publish_txn(tablet->data_dir()->get_meta(), partition_id, transaction_id, - tablet->tablet_id(), tablet->schema_hash(), tablet->tablet_uid(), version, - stats); + tablet->tablet_id(), tablet->tablet_uid(), version, stats); } // delete the txn from manager if it is not committed(not have a valid rowset) Status TxnManager::rollback_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, TTransactionId transaction_id) { - return rollback_txn(partition_id, transaction_id, tablet->tablet_id(), tablet->schema_hash(), - tablet->tablet_uid()); + return rollback_txn(partition_id, transaction_id, tablet->tablet_id(), tablet->tablet_uid()); } Status TxnManager::delete_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, TTransactionId transaction_id) { return delete_txn(tablet->data_dir()->get_meta(), partition_id, transaction_id, - tablet->tablet_id(), tablet->schema_hash(), tablet->tablet_uid()); + tablet->tablet_id(), tablet->tablet_uid()); } void TxnManager::set_txn_related_delete_bitmap(TPartitionId partition_id, TTransactionId transaction_id, TTabletId tablet_id, - SchemaHash schema_hash, TabletUid tablet_uid, - bool unique_key_merge_on_write, + TabletUid tablet_uid, bool unique_key_merge_on_write, DeleteBitmapPtr delete_bitmap, const RowsetIdUnorderedSet& rowset_ids) { pair key(partition_id, transaction_id); - TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); + TabletInfo tablet_info(tablet_id, tablet_uid); std::lock_guard txn_lock(_get_txn_lock(transaction_id)); { @@ -215,16 +209,15 @@ void TxnManager::set_txn_related_delete_bitmap(TPartitionId partition_id, Status TxnManager::commit_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id, TTabletId tablet_id, - SchemaHash schema_hash, TabletUid tablet_uid, - const PUniqueId& load_id, const RowsetSharedPtr& rowset_ptr, - bool is_recovery) { + TabletUid tablet_uid, const PUniqueId& load_id, + const RowsetSharedPtr& rowset_ptr, bool is_recovery) { if (partition_id < 1 || transaction_id < 1 || tablet_id < 1) { LOG(FATAL) << "invalid commit req " << " partition_id=" << partition_id << " transaction_id=" << transaction_id << " tablet_id=" << tablet_id; } pair key(partition_id, transaction_id); - TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); + TabletInfo tablet_info(tablet_id, tablet_uid); if (rowset_ptr == nullptr) { return Status::Error( "could not commit txn because rowset ptr is null. partition_id: {}, " @@ -316,7 +309,7 @@ Status TxnManager::commit_txn(OlapMeta* meta, TPartitionId partition_id, // remove a txn from txn manager Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id, TTabletId tablet_id, - SchemaHash schema_hash, TabletUid tablet_uid, const Version& version, + TabletUid tablet_uid, const Version& version, TabletPublishStatistics* stats) { auto tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id); if (tablet == nullptr) { @@ -325,7 +318,7 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, DCHECK(stats != nullptr); pair key(partition_id, transaction_id); - TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); + TabletInfo tablet_info(tablet_id, tablet_uid); RowsetSharedPtr rowset = nullptr; TabletTxnInfo tablet_txn_info; int64_t t1 = MonotonicMicros(); @@ -447,9 +440,9 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, // may be committed in another thread and our current thread meets errors when writing to data file // BE has to wait for fe call clear txn api Status TxnManager::rollback_txn(TPartitionId partition_id, TTransactionId transaction_id, - TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid) { + TTabletId tablet_id, TabletUid tablet_uid) { pair key(partition_id, transaction_id); - TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); + TabletInfo tablet_info(tablet_id, tablet_uid); std::lock_guard wrlock(_get_txn_map_lock(transaction_id)); txn_tablet_map_t& txn_tablet_map = _get_txn_tablet_map(transaction_id); auto it = txn_tablet_map.find(key); @@ -481,9 +474,9 @@ Status TxnManager::rollback_txn(TPartitionId partition_id, TTransactionId transa // could not delete the rowset if it already has a valid version Status TxnManager::delete_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id, TTabletId tablet_id, - SchemaHash schema_hash, TabletUid tablet_uid) { + TabletUid tablet_uid) { pair key(partition_id, transaction_id); - TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); + TabletInfo tablet_info(tablet_id, tablet_uid); std::lock_guard txn_wrlock(_get_txn_map_lock(transaction_id)); txn_tablet_map_t& txn_tablet_map = _get_txn_tablet_map(transaction_id); auto it = txn_tablet_map.find(key); @@ -526,15 +519,15 @@ Status TxnManager::delete_txn(OlapMeta* meta, TPartitionId partition_id, return Status::OK(); } -void TxnManager::get_tablet_related_txns(TTabletId tablet_id, SchemaHash schema_hash, - TabletUid tablet_uid, int64_t* partition_id, +void TxnManager::get_tablet_related_txns(TTabletId tablet_id, TabletUid tablet_uid, + int64_t* partition_id, std::set* transaction_ids) { if (partition_id == nullptr || transaction_ids == nullptr) { LOG(WARNING) << "parameter is null when get transactions by tablet"; return; } - TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); + TabletInfo tablet_info(tablet_id, tablet_uid); for (int32_t i = 0; i < _txn_map_shard_size; i++) { std::shared_lock txn_rdlock(_txn_map_locks[i]); txn_tablet_map_t& txn_tablet_map = _txn_tablet_maps[i]; @@ -554,8 +547,8 @@ void TxnManager::get_tablet_related_txns(TTabletId tablet_id, SchemaHash schema_ // force drop all txns related with the tablet // maybe lock error, because not get txn lock before remove from meta void TxnManager::force_rollback_tablet_related_txns(OlapMeta* meta, TTabletId tablet_id, - SchemaHash schema_hash, TabletUid tablet_uid) { - TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); + TabletUid tablet_uid) { + TabletInfo tablet_info(tablet_id, tablet_uid); for (int32_t i = 0; i < _txn_map_shard_size; i++) { std::lock_guard txn_wrlock(_txn_map_locks[i]); txn_tablet_map_t& txn_tablet_map = _txn_tablet_maps[i]; @@ -646,9 +639,9 @@ void TxnManager::get_all_commit_tablet_txn_info_by_tablet( } bool TxnManager::has_txn(TPartitionId partition_id, TTransactionId transaction_id, - TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid) { + TTabletId tablet_id, TabletUid tablet_uid) { pair key(partition_id, transaction_id); - TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); + TabletInfo tablet_info(tablet_id, tablet_uid); std::shared_lock txn_rdlock(_get_txn_map_lock(transaction_id)); txn_tablet_map_t& txn_tablet_map = _get_txn_tablet_map(transaction_id); auto it = txn_tablet_map.find(key); diff --git a/be/src/olap/txn_manager.h b/be/src/olap/txn_manager.h index c7f781a58a7afb2..c311fed87994616 100644 --- a/be/src/olap/txn_manager.h +++ b/be/src/olap/txn_manager.h @@ -115,8 +115,8 @@ class TxnManager { bool is_ingest = false); // most used for ut Status prepare_txn(TPartitionId partition_id, TTransactionId transaction_id, - TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, - const PUniqueId& load_id, bool is_ingest = false); + TTabletId tablet_id, TabletUid tablet_uid, const PUniqueId& load_id, + bool is_ingest = false); Status commit_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, TTransactionId transaction_id, const PUniqueId& load_id, @@ -134,28 +134,27 @@ class TxnManager { TTransactionId transaction_id); Status commit_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id, - TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, - const PUniqueId& load_id, const RowsetSharedPtr& rowset_ptr, - bool is_recovery); + TTabletId tablet_id, TabletUid tablet_uid, const PUniqueId& load_id, + const RowsetSharedPtr& rowset_ptr, bool is_recovery); // remove a txn from txn manager // not persist rowset meta because Status publish_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id, - TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, - const Version& version, TabletPublishStatistics* stats); + TTabletId tablet_id, TabletUid tablet_uid, const Version& version, + TabletPublishStatistics* stats); // delete the txn from manager if it is not committed(not have a valid rowset) Status rollback_txn(TPartitionId partition_id, TTransactionId transaction_id, - TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid); + TTabletId tablet_id, TabletUid tablet_uid); // remove the txn from txn manager // delete the related rowset if it is not null // delete rowset related data if it is not null Status delete_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id, - TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid); + TTabletId tablet_id, TabletUid tablet_uid); - void get_tablet_related_txns(TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, - int64_t* partition_id, std::set* transaction_ids); + void get_tablet_related_txns(TTabletId tablet_id, TabletUid tablet_uid, int64_t* partition_id, + std::set* transaction_ids); void get_txn_related_tablets(const TTransactionId transaction_id, TPartitionId partition_ids, std::map* tablet_infos); @@ -164,14 +163,14 @@ class TxnManager { // Just check if the txn exists. bool has_txn(TPartitionId partition_id, TTransactionId transaction_id, TTabletId tablet_id, - SchemaHash schema_hash, TabletUid tablet_uid); + TabletUid tablet_uid); // Get all expired txns and save them in expire_txn_map. // This is currently called before reporting all tablet info, to avoid iterating txn map for every tablets. void build_expire_txn_map(std::map>* expire_txn_map); void force_rollback_tablet_related_txns(OlapMeta* meta, TTabletId tablet_id, - SchemaHash schema_hash, TabletUid tablet_uid); + TabletUid tablet_uid); void get_partition_ids(const TTransactionId transaction_id, std::vector* partition_ids); @@ -183,8 +182,8 @@ class TxnManager { bool is_succeed); void set_txn_related_delete_bitmap(TPartitionId partition_id, TTransactionId transaction_id, - TTabletId tablet_id, SchemaHash schema_hash, - TabletUid tablet_uid, bool unique_key_merge_on_write, + TTabletId tablet_id, TabletUid tablet_uid, + bool unique_key_merge_on_write, DeleteBitmapPtr delete_bitmap, const RowsetIdUnorderedSet& rowset_ids); void get_all_commit_tablet_txn_info_by_tablet( diff --git a/be/src/olap/types.cpp b/be/src/olap/types.cpp index e179e59b18c852f..102f00c7e1ef413 100644 --- a/be/src/olap/types.cpp +++ b/be/src/olap/types.cpp @@ -190,7 +190,7 @@ const TypeInfo* get_struct_type_info(std::vector field_types) { // TODO: Support the type info of the nested array with more than 9 depths. // TODO(xy): Support the type info of the nested struct -TypeInfoPtr get_type_info(segment_v2::ColumnMetaPB* column_meta_pb) { +TypeInfoPtr get_type_info(const segment_v2::ColumnMetaPB* column_meta_pb) { FieldType type = (FieldType)column_meta_pb->type(); if (UNLIKELY(type == FieldType::OLAP_FIELD_TYPE_STRUCT)) { std::vector field_types; @@ -200,14 +200,10 @@ TypeInfoPtr get_type_info(segment_v2::ColumnMetaPB* column_meta_pb) { } return create_dynamic_type_info_ptr(get_struct_type_info(field_types)); } else if (UNLIKELY(type == FieldType::OLAP_FIELD_TYPE_ARRAY)) { - int32_t iterations = 0; - const auto* child_column = &column_meta_pb->children_columns(0); - while (child_column->type() == int(FieldType::OLAP_FIELD_TYPE_ARRAY)) { - iterations++; - child_column = &child_column->children_columns(0); - } - return create_static_type_info_ptr( - get_array_type_info((FieldType)child_column->type(), iterations)); + segment_v2::ColumnMetaPB child_column = column_meta_pb->children_columns(0); + TypeInfoPtr child_info = get_type_info(&child_column); + ArrayTypeInfo* array_type_info = new ArrayTypeInfo(std::move(child_info)); + return create_dynamic_type_info_ptr(array_type_info); } else if (UNLIKELY(type == FieldType::OLAP_FIELD_TYPE_MAP)) { segment_v2::ColumnMetaPB key_meta = column_meta_pb->children_columns(0); TypeInfoPtr key_type_info = get_type_info(&key_meta); @@ -216,7 +212,7 @@ TypeInfoPtr get_type_info(segment_v2::ColumnMetaPB* column_meta_pb) { MapTypeInfo* map_type_info = new MapTypeInfo(std::move(key_type_info), std::move(value_type_info)); - return create_static_type_info_ptr(map_type_info); + return create_dynamic_type_info_ptr(map_type_info); } else { return create_static_type_info_ptr(get_scalar_type_info(type)); } @@ -249,13 +245,10 @@ TypeInfoPtr get_type_info(const TabletColumn* col) { } return create_dynamic_type_info_ptr(get_struct_type_info(field_types)); } else if (UNLIKELY(type == FieldType::OLAP_FIELD_TYPE_ARRAY)) { - int32_t iterations = 0; const auto* child_column = &col->get_sub_column(0); - while (child_column->type() == FieldType::OLAP_FIELD_TYPE_ARRAY) { - iterations++; - child_column = &child_column->get_sub_column(0); - } - return create_static_type_info_ptr(get_array_type_info(child_column->type(), iterations)); + TypeInfoPtr item_type = get_type_info(child_column); + ArrayTypeInfo* array_type_info = new ArrayTypeInfo(std::move(item_type)); + return create_dynamic_type_info_ptr(array_type_info); } else if (UNLIKELY(type == FieldType::OLAP_FIELD_TYPE_MAP)) { const auto* key_column = &col->get_sub_column(0); TypeInfoPtr key_type = get_type_info(key_column); diff --git a/be/src/olap/types.h b/be/src/olap/types.h index 9ef6522524ecbdc..40d812a1605e722 100644 --- a/be/src/olap/types.h +++ b/be/src/olap/types.h @@ -607,7 +607,7 @@ bool is_scalar_type(FieldType field_type); const TypeInfo* get_scalar_type_info(FieldType field_type); -TypeInfoPtr get_type_info(segment_v2::ColumnMetaPB* column_meta_pb); +TypeInfoPtr get_type_info(const segment_v2::ColumnMetaPB* column_meta_pb); TypeInfoPtr get_type_info(const TabletColumn* col); diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index a20522f149a9724..38323cad2faa47e 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -20,6 +20,7 @@ #include #include "pipeline/exec/operator.h" +#include "pipeline/exec/streaming_aggregation_sink_operator.h" #include "runtime/primitive_type.h" namespace doris::pipeline { @@ -45,8 +46,9 @@ OPERATOR_CODE_GENERATOR(AggSinkOperator, StreamingOperator) /// using the planner's estimated input cardinality and the assumption that input /// is in a random order. This means that we assume that the reduction factor will /// increase over time. -AggSinkLocalState::AggSinkLocalState(DataSinkOperatorX* parent, RuntimeState* state) - : PipelineXSinkLocalState(parent, state), +template +AggSinkLocalState::AggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : PipelineXSinkLocalState(parent, state), _hash_table_compute_timer(nullptr), _hash_table_input_counter(nullptr), _build_timer(nullptr), @@ -58,12 +60,12 @@ AggSinkLocalState::AggSinkLocalState(DataSinkOperatorX* parent, RuntimeState* st _deserialize_data_timer(nullptr), _max_row_size_counter(nullptr) {} -Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - _dependency = (AggDependency*)info.dependency; - _shared_state = (AggSharedState*)_dependency->shared_state(); +template +Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); _agg_data = _shared_state->agg_data.get(); _agg_arena_pool = _shared_state->agg_arena_pool.get(); - auto& p = _parent->cast(); + auto& p = _parent->cast(); _dependency->set_align_aggregate_states(p._align_aggregate_states); _dependency->set_total_size_of_aggregate_states(p._total_size_of_aggregate_states); _dependency->set_offsets_of_aggregate_states(p._offsets_of_aggregate_states); @@ -78,7 +80,6 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { for (size_t i = 0; i < _shared_state->probe_expr_ctxs.size(); i++) { RETURN_IF_ERROR(p._probe_expr_ctxs[i]->clone(state, _shared_state->probe_expr_ctxs[i])); } - _profile = p._pool->add(new RuntimeProfile("AggSinkLocalState")); _memory_usage_counter = ADD_LABEL_COUNTER(profile(), "MemoryUsage"); _hash_table_memory_usage = ADD_CHILD_COUNTER(profile(), "HashTable", TUnit::BYTES, "MemoryUsage"); @@ -108,15 +109,15 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { _shared_state->agg_profile_arena->alloc(p._total_size_of_aggregate_states)); if (p._is_merge) { - _executor.execute = std::bind(&AggSinkLocalState::_merge_without_key, this, - std::placeholders::_1); + _executor.execute = std::bind(&AggSinkLocalState::_merge_without_key, + this, std::placeholders::_1); } else { - _executor.execute = std::bind(&AggSinkLocalState::_execute_without_key, this, - std::placeholders::_1); + _executor.execute = std::bind(&AggSinkLocalState::_execute_without_key, + this, std::placeholders::_1); } _executor.update_memusage = - std::bind(&AggSinkLocalState::_update_memusage_without_key, this); + std::bind(&AggSinkLocalState::_update_memusage_without_key, this); } else { _init_hash_method(_shared_state->probe_expr_ctxs); @@ -135,15 +136,17 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { }, _agg_data->method_variant); if (p._is_merge) { - _executor.execute = std::bind(&AggSinkLocalState::_merge_with_serialized_key, - this, std::placeholders::_1); + _executor.execute = + std::bind(&AggSinkLocalState::_merge_with_serialized_key, this, + std::placeholders::_1); } else { - _executor.execute = std::bind(&AggSinkLocalState::_execute_with_serialized_key, - this, std::placeholders::_1); + _executor.execute = + std::bind(&AggSinkLocalState::_execute_with_serialized_key, + this, std::placeholders::_1); } - _executor.update_memusage = - std::bind(&AggSinkLocalState::_update_memusage_with_serialized_key, this); + _executor.update_memusage = std::bind( + &AggSinkLocalState::_update_memusage_with_serialized_key, this); _should_limit_output = p._limit != -1 && // has limit (!p._have_conjuncts) && // no having conjunct @@ -159,20 +162,22 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { return Status::OK(); } -Status AggSinkLocalState::_execute_without_key(vectorized::Block* block) { +template +Status AggSinkLocalState::_execute_without_key(vectorized::Block* block) { DCHECK(_agg_data->without_key != nullptr); SCOPED_TIMER(_build_timer); for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_single_add( block, _agg_data->without_key + - _parent->cast()._offsets_of_aggregate_states[i], + _parent->cast()._offsets_of_aggregate_states[i], _agg_arena_pool)); } return Status::OK(); } -Status AggSinkLocalState::_merge_with_serialized_key(vectorized::Block* block) { +template +Status AggSinkLocalState::_merge_with_serialized_key(vectorized::Block* block) { if (_reach_limit) { return _merge_with_serialized_key_helper(block); } else { @@ -180,7 +185,8 @@ Status AggSinkLocalState::_merge_with_serialized_key(vectorized::Block* block) { } } -size_t AggSinkLocalState::_memory_usage() const { +template +size_t AggSinkLocalState::_memory_usage() const { size_t usage = 0; std::visit( [&](auto&& agg_method) { @@ -204,7 +210,8 @@ size_t AggSinkLocalState::_memory_usage() const { return usage; } -void AggSinkLocalState::_update_memusage_with_serialized_key() { +template +void AggSinkLocalState::_update_memusage_with_serialized_key() { std::visit( [&](auto&& agg_method) -> void { auto& data = agg_method.data; @@ -226,8 +233,9 @@ void AggSinkLocalState::_update_memusage_with_serialized_key() { _agg_data->method_variant); } +template template -Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* block) { +Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* block) { SCOPED_TIMER(_merge_timer); size_t key_size = _shared_state->probe_expr_ctxs.size(); @@ -272,7 +280,7 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b ->function() ->deserialize_and_merge_vec_selected( _places.data(), - _parent->cast() + _parent->cast() ._offsets_of_aggregate_states[i], _deserialize_buffer.data(), (vectorized::ColumnString*)(column.get()), _agg_arena_pool, @@ -280,7 +288,8 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b } } else { RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_batch_add_selected( - block, _parent->cast()._offsets_of_aggregate_states[i], + block, + _parent->cast()._offsets_of_aggregate_states[i], _places.data(), _agg_arena_pool)); } } @@ -310,19 +319,22 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b SCOPED_TIMER(_deserialize_data_timer); _shared_state->aggregate_evaluators[i]->function()->deserialize_and_merge_vec( _places.data(), - _parent->cast()._offsets_of_aggregate_states[i], + _parent->cast() + ._offsets_of_aggregate_states[i], _deserialize_buffer.data(), (vectorized::ColumnString*)(column.get()), _agg_arena_pool, rows); } } else { RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_batch_add( - block, _parent->cast()._offsets_of_aggregate_states[i], + block, + _parent->cast()._offsets_of_aggregate_states[i], _places.data(), _agg_arena_pool)); } } if (_should_limit_output) { - _reach_limit = _get_hash_table_size() >= _parent->cast()._limit; + _reach_limit = + _get_hash_table_size() >= _parent->cast()._limit; } } @@ -332,7 +344,8 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b // We should call this function only at 1st phase. // 1st phase: is_merge=true, only have one SlotRef. // 2nd phase: is_merge=false, maybe have multiple exprs. -int AggSinkLocalState::_get_slot_column_id(const vectorized::AggFnEvaluator* evaluator) { +template +int AggSinkLocalState::_get_slot_column_id(const vectorized::AggFnEvaluator* evaluator) { auto ctxs = evaluator->input_exprs_ctxs(); CHECK(ctxs.size() == 1 && ctxs[0]->root()->is_slot_ref()) << "input_exprs_ctxs is invalid, input_exprs_ctx[0]=" @@ -340,7 +353,8 @@ int AggSinkLocalState::_get_slot_column_id(const vectorized::AggFnEvaluator* eva return ((vectorized::VSlotRef*)ctxs[0]->root().get())->column_id(); } -Status AggSinkLocalState::_merge_without_key(vectorized::Block* block) { +template +Status AggSinkLocalState::_merge_without_key(vectorized::Block* block) { SCOPED_TIMER(_merge_timer); DCHECK(_agg_data->without_key != nullptr); for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { @@ -353,21 +367,22 @@ Status AggSinkLocalState::_merge_without_key(vectorized::Block* block) { SCOPED_TIMER(_deserialize_data_timer); _shared_state->aggregate_evaluators[i]->function()->deserialize_and_merge_from_column( - _agg_data->without_key + - _parent->cast()._offsets_of_aggregate_states[i], + _agg_data->without_key + _parent->cast() + ._offsets_of_aggregate_states[i], *column, _agg_arena_pool); } else { RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_single_add( block, - _agg_data->without_key + - _parent->cast()._offsets_of_aggregate_states[i], + _agg_data->without_key + _parent->cast() + ._offsets_of_aggregate_states[i], _agg_arena_pool)); } } return Status::OK(); } -void AggSinkLocalState::_update_memusage_without_key() { +template +void AggSinkLocalState::_update_memusage_without_key() { auto arena_memory_usage = _agg_arena_pool->size() - _dependency->mem_usage_record().used_in_arena; _dependency->mem_tracker()->consume(arena_memory_usage); @@ -375,7 +390,8 @@ void AggSinkLocalState::_update_memusage_without_key() { _dependency->mem_usage_record().used_in_arena = _agg_arena_pool->size(); } -Status AggSinkLocalState::_execute_with_serialized_key(vectorized::Block* block) { +template +Status AggSinkLocalState::_execute_with_serialized_key(vectorized::Block* block) { if (_reach_limit) { return _execute_with_serialized_key_helper(block); } else { @@ -383,8 +399,9 @@ Status AggSinkLocalState::_execute_with_serialized_key(vectorized::Block* block) } } +template template -Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorized::Block* block) { +Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorized::Block* block) { SCOPED_TIMER(_build_timer); DCHECK(!_shared_state->probe_expr_ctxs.empty()); @@ -412,7 +429,8 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorized::Block* for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_batch_add_selected( - block, _parent->cast()._offsets_of_aggregate_states[i], + block, + _parent->cast()._offsets_of_aggregate_states[i], _places.data(), _agg_arena_pool)); } } else { @@ -420,13 +438,15 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorized::Block* for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_batch_add( - block, _parent->cast()._offsets_of_aggregate_states[i], + block, + _parent->cast()._offsets_of_aggregate_states[i], _places.data(), _agg_arena_pool)); } if (_should_limit_output) { - _reach_limit = _get_hash_table_size() >= _parent->cast()._limit; - if (_reach_limit && _parent->cast()._can_short_circuit) { + _reach_limit = + _get_hash_table_size() >= _parent->cast()._limit; + if (_reach_limit && _parent->cast()._can_short_circuit) { _dependency->set_done(); return Status::Error(""); } @@ -436,14 +456,16 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorized::Block* return Status::OK(); } -size_t AggSinkLocalState::_get_hash_table_size() { +template +size_t AggSinkLocalState::_get_hash_table_size() { return std::visit([&](auto&& agg_method) { return agg_method.data.size(); }, _agg_data->method_variant); } -void AggSinkLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* places, - vectorized::ColumnRawPtrs& key_columns, - const size_t num_rows) { +template +void AggSinkLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* places, + vectorized::ColumnRawPtrs& key_columns, + const size_t num_rows) { std::visit( [&](auto&& agg_method) -> void { SCOPED_TIMER(_hash_table_compute_timer); @@ -454,7 +476,7 @@ void AggSinkLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* p _pre_serialize_key_if_need(state, agg_method, key_columns, num_rows); - auto creator = [this](const auto& ctor, const auto& key) { + auto creator = [&](const auto& ctor, const auto& key) { using KeyType = std::decay_t; if constexpr (HashTableTraits::is_string_hash_table && !std::is_same_v) { @@ -472,10 +494,11 @@ void AggSinkLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* p } }; - auto creator_for_null_key = [this](auto& mapped) { + auto creator_for_null_key = [&](auto& mapped) { mapped = _agg_arena_pool->aligned_alloc( - _parent->cast()._total_size_of_aggregate_states, - _parent->cast()._align_aggregate_states); + _parent->cast() + ._total_size_of_aggregate_states, + _parent->cast()._align_aggregate_states); _dependency->create_agg_status(mapped); }; @@ -526,9 +549,10 @@ void AggSinkLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* p _agg_data->method_variant); } -void AggSinkLocalState::_find_in_hash_table(vectorized::AggregateDataPtr* places, - vectorized::ColumnRawPtrs& key_columns, - size_t num_rows) { +template +void AggSinkLocalState::_find_in_hash_table(vectorized::AggregateDataPtr* places, + vectorized::ColumnRawPtrs& key_columns, + size_t num_rows) { std::visit( [&](auto&& agg_method) -> void { using HashMethodType = std::decay_t; @@ -579,7 +603,9 @@ void AggSinkLocalState::_find_in_hash_table(vectorized::AggregateDataPtr* places _agg_data->method_variant); } -void AggSinkLocalState::_init_hash_method(const vectorized::VExprContextSPtrs& probe_exprs) { +template +void AggSinkLocalState::_init_hash_method( + const vectorized::VExprContextSPtrs& probe_exprs) { DCHECK(probe_exprs.size() >= 1); using Type = vectorized::AggregatedDataVariants::Type; @@ -633,9 +659,9 @@ void AggSinkLocalState::_init_hash_method(const vectorized::VExprContextSPtrs& p t = Type::serialized; } - _agg_data->init( - get_hash_key_type_with_phase(t, !_parent->cast()._is_first_phase), - is_nullable); + _agg_data->init(get_hash_key_type_with_phase( + t, !_parent->cast()._is_first_phase), + is_nullable); } else { bool use_fixed_key = true; bool has_null = false; @@ -678,7 +704,7 @@ void AggSinkLocalState::_init_hash_method(const vectorized::VExprContextSPtrs& p t = Type::int256_keys; } _agg_data->init(get_hash_key_type_with_phase( - t, !_parent->cast()._is_first_phase), + t, !_parent->cast()._is_first_phase), has_null); } else { _agg_data->init(Type::serialized); @@ -686,16 +712,16 @@ void AggSinkLocalState::_init_hash_method(const vectorized::VExprContextSPtrs& p } } -Status AggSinkLocalState::try_spill_disk(bool eos) { - if (_parent->cast()._external_agg_bytes_threshold == 0) { +template +Status AggSinkLocalState::try_spill_disk(bool eos) { + if (_parent->cast()._external_agg_bytes_threshold == 0) { return Status::OK(); } return std::visit( [&](auto&& agg_method) -> Status { auto& hash_table = agg_method.data; - if (!eos && - _memory_usage() < - _parent->cast()._external_agg_bytes_threshold) { + if (!eos && _memory_usage() < _parent->cast() + ._external_agg_bytes_threshold) { return Status::OK(); } @@ -709,9 +735,10 @@ Status AggSinkLocalState::try_spill_disk(bool eos) { _agg_data->method_variant); } -AggSinkOperatorX::AggSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs) - : DataSinkOperatorX(tnode.node_id), +template +AggSinkOperatorX::AggSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, + const DescriptorTbl& descs) + : DataSinkOperatorX(tnode.node_id), _intermediate_tuple_id(tnode.agg_node.intermediate_tuple_id), _intermediate_tuple_desc(nullptr), _output_tuple_id(tnode.agg_node.output_tuple_id), @@ -722,10 +749,11 @@ AggSinkOperatorX::AggSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, _limit(tnode.limit), _have_conjuncts(tnode.__isset.vconjunct && !tnode.vconjunct.nodes.empty()) { _is_first_phase = tnode.agg_node.__isset.is_first_phase && tnode.agg_node.is_first_phase; - _name = "AggSinkOperatorX"; } -Status AggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { +template +Status AggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(DataSinkOperatorX::init(tnode, state)); // ignore return status for now , so we need to introduce ExecNode::init() RETURN_IF_ERROR( vectorized::VExpr::create_expr_trees(tnode.agg_node.grouping_exprs, _probe_expr_ctxs)); @@ -763,11 +791,13 @@ Status AggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { return Status::OK(); } -Status AggSinkOperatorX::prepare(RuntimeState* state) { +template +Status AggSinkOperatorX::prepare(RuntimeState* state) { _intermediate_tuple_desc = state->desc_tbl().get_tuple_descriptor(_intermediate_tuple_id); _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_output_tuple_id); DCHECK_EQ(_intermediate_tuple_desc->slots().size(), _output_tuple_desc->slots().size()); - RETURN_IF_ERROR(vectorized::VExpr::prepare(_probe_expr_ctxs, state, _child_x->row_desc())); + RETURN_IF_ERROR(vectorized::VExpr::prepare( + _probe_expr_ctxs, state, DataSinkOperatorX::_child_x->row_desc())); int j = _probe_expr_ctxs.size(); for (int i = 0; i < j; ++i) { @@ -782,7 +812,8 @@ Status AggSinkOperatorX::prepare(RuntimeState* state) { SlotDescriptor* intermediate_slot_desc = _intermediate_tuple_desc->slots()[j]; SlotDescriptor* output_slot_desc = _output_tuple_desc->slots()[j]; RETURN_IF_ERROR(_aggregate_evaluators[i]->prepare( - state, _child_x->row_desc(), intermediate_slot_desc, output_slot_desc)); + state, DataSinkOperatorX::_child_x->row_desc(), + intermediate_slot_desc, output_slot_desc)); } _offsets_of_aggregate_states.resize(_aggregate_evaluators.size()); @@ -818,11 +849,12 @@ Status AggSinkOperatorX::prepare(RuntimeState* state) { _is_merge ? "true" : "false", _needs_finalize ? "true" : "false", std::to_string(_aggregate_evaluators.size()), std::to_string(_limit)); std::string title = fmt::format("Aggregation Sink {}", fmt::to_string(msg)); - _profile = _pool->add(new RuntimeProfile(title)); + DataSinkOperatorX::_profile = _pool->add(new RuntimeProfile(title)); return Status::OK(); } -Status AggSinkOperatorX::open(RuntimeState* state) { +template +Status AggSinkOperatorX::open(RuntimeState* state) { RETURN_IF_ERROR(vectorized::VExpr::open(_probe_expr_ctxs, state)); for (int i = 0; i < _aggregate_evaluators.size(); ++i) { @@ -833,9 +865,11 @@ Status AggSinkOperatorX::open(RuntimeState* state) { return Status::OK(); } -Status AggSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in_block, - SourceState source_state) { - auto& local_state = state->get_sink_local_state(id())->cast(); +template +Status AggSinkOperatorX::sink(doris::RuntimeState* state, + vectorized::Block* in_block, + SourceState source_state) { + auto& local_state = state->get_sink_local_state(id())->template cast(); local_state._shared_state->input_num_rows += in_block->rows(); if (in_block->rows() > 0) { RETURN_IF_ERROR(local_state._executor.execute(in_block)); @@ -852,13 +886,8 @@ Status AggSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in_ return Status::OK(); } -Status AggSinkOperatorX::setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) { - auto local_state = AggSinkLocalState::create_shared(this, state); - state->emplace_sink_local_state(id(), local_state); - return local_state->init(state, info); -} - -Status AggSinkLocalState::close(RuntimeState* state) { +template +Status AggSinkLocalState::close(RuntimeState* state) { if (_closed) { return Status::OK(); } @@ -871,7 +900,14 @@ Status AggSinkLocalState::close(RuntimeState* state) { std::vector tmp_hash_values; _hash_values.swap(tmp_hash_values); - return PipelineXSinkLocalState::close(state); + return PipelineXSinkLocalState::close(state); } -} // namespace doris::pipeline \ No newline at end of file +class StreamingAggSinkLocalState; + +template class AggSinkOperatorX; +template class AggSinkOperatorX; +template class AggSinkLocalState; +template class AggSinkLocalState; + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h index 477ad08735b36b2..3237da0b6b8327a 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.h +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -21,6 +21,7 @@ #include "operator.h" #include "pipeline/pipeline_x/dependency.h" +#include "pipeline/pipeline_x/operator.h" #include "runtime/block_spill_manager.h" #include "runtime/exec_env.h" #include "vec/exec/vaggregation_node.h" @@ -44,13 +45,13 @@ class AggSinkOperator final : public StreamingOperator { bool can_write() override { return true; } }; +template class AggSinkOperatorX; -class AggSinkLocalState : public PipelineXSinkLocalState { - ENABLE_FACTORY_CREATOR(AggSinkLocalState); - +template +class AggSinkLocalState : public PipelineXSinkLocalState { public: - AggSinkLocalState(DataSinkOperatorX* parent, RuntimeState* state); + virtual ~AggSinkLocalState() = default; virtual Status init(RuntimeState* state, LocalSinkStateInfo& info) override; virtual Status close(RuntimeState* state) override; @@ -58,6 +59,9 @@ class AggSinkLocalState : public PipelineXSinkLocalState { Status try_spill_disk(bool eos = false); protected: + AggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); + + template friend class AggSinkOperatorX; Status _execute_without_key(vectorized::Block* block); @@ -293,9 +297,7 @@ class AggSinkLocalState : public PipelineXSinkLocalState { vectorized::Block _preagg_block = vectorized::Block(); - AggDependency* _dependency; vectorized::AggregatedDataVariants* _agg_data; - AggSharedState* _shared_state; vectorized::Arena* _agg_arena_pool; std::vector _hash_values; @@ -310,29 +312,40 @@ class AggSinkLocalState : public PipelineXSinkLocalState { executor _executor; }; -class AggSinkOperatorX : public DataSinkOperatorX { +class BlockingAggSinkLocalState : public AggSinkLocalState { +public: + ENABLE_FACTORY_CREATOR(BlockingAggSinkLocalState); + using Parent = AggSinkOperatorX; + + BlockingAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : AggSinkLocalState(parent, state) {} + ~BlockingAggSinkLocalState() = default; +}; + +template +class AggSinkOperatorX : public DataSinkOperatorX { public: AggSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + virtual ~AggSinkOperatorX() = default; Status init(const TDataSink& tsink) override { - return Status::InternalError("{} should not init with TPlanNode", _name); + return Status::InternalError("{} should not init with TPlanNode", + DataSinkOperatorX::_name); } Status init(const TPlanNode& tnode, RuntimeState* state) override; Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; - virtual Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) override; virtual Status sink(RuntimeState* state, vectorized::Block* in_block, SourceState source_state) override; virtual bool can_write(RuntimeState* state) override { return true; } - void get_dependency(DependencySPtr& dependency) override { - dependency.reset(new AggDependency(id())); - } + using DataSinkOperatorX::id; -private: +protected: + template friend class AggSinkLocalState; friend class StreamingAggSinkLocalState; std::vector _aggregate_evaluators; diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index cbee5a832fa0708..054df663decaa6b 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -28,7 +28,7 @@ namespace pipeline { OPERATOR_CODE_GENERATOR(AggSourceOperator, SourceOperator) AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) - : PipelineXLocalState(state, parent), + : PipelineXLocalState(state, parent), _get_results_timer(nullptr), _serialize_result_timer(nullptr), _hash_table_iterate_timer(nullptr), @@ -37,9 +37,7 @@ AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) _hash_table_size_counter(nullptr) {} Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { - RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); - _dependency = (AggDependency*)info.dependency; - _shared_state = (AggSharedState*)_dependency->shared_state(); + RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); _agg_data = _shared_state->agg_data.get(); _get_results_timer = ADD_TIMER(profile(), "GetResultsTime"); _serialize_result_timer = ADD_TIMER(profile(), "SerializeResultTime"); @@ -493,8 +491,8 @@ Status AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::B } AggSourceOperatorX::AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs, std::string op_name) - : OperatorXBase(pool, tnode, descs, op_name), + const DescriptorTbl& descs) + : OperatorX(pool, tnode, descs), _needs_finalize(tnode.agg_node.need_finalize), _without_key(tnode.agg_node.grouping_exprs.empty()) {} @@ -547,13 +545,7 @@ Status AggLocalState::close(RuntimeState* state) { std::vector tmp_values; _shared_state->values.swap(tmp_values); - return PipelineXLocalState::close(state); -} - -Status AggSourceOperatorX::setup_local_state(RuntimeState* state, LocalStateInfo& info) { - auto local_state = AggLocalState::create_shared(state, this); - state->emplace_local_state(id(), local_state); - return local_state->init(state, info); + return PipelineXLocalState::close(state); } bool AggSourceOperatorX::can_read(RuntimeState* state) { diff --git a/be/src/pipeline/exec/aggregation_source_operator.h b/be/src/pipeline/exec/aggregation_source_operator.h index e974d52f54d99ab..80a104a410f77dc 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.h +++ b/be/src/pipeline/exec/aggregation_source_operator.h @@ -21,6 +21,7 @@ #include "common/status.h" #include "operator.h" #include "pipeline/pipeline_x/dependency.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/vaggregation_node.h" namespace doris { @@ -48,7 +49,7 @@ class AggSourceOperator final : public SourceOperator }; class AggSourceOperatorX; -class AggLocalState final : public PipelineXLocalState { +class AggLocalState final : public PipelineXLocalState { public: ENABLE_FACTORY_CREATOR(AggLocalState); AggLocalState(RuntimeState* state, OperatorXBase* parent); @@ -102,20 +103,16 @@ class AggLocalState final : public PipelineXLocalState { executor _executor; - AggDependency* _dependency; - AggSharedState* _shared_state; vectorized::AggregatedDataVariants* _agg_data; bool _agg_data_created_without_key = false; }; -class AggSourceOperatorX : public OperatorXBase { +class AggSourceOperatorX : public OperatorX { public: - AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string op_name); + AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + virtual ~AggSourceOperatorX() = default; virtual bool can_read(RuntimeState* state) override; - Status setup_local_state(RuntimeState* state, LocalStateInfo& info) override; - virtual Status get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) override; diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index 1ebe342e85f1a8c..75cc6e336ad60d3 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -27,15 +27,11 @@ namespace doris::pipeline { OPERATOR_CODE_GENERATOR(AnalyticSinkOperator, StreamingOperator) Status AnalyticSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); - _mem_tracker = std::make_unique("ExchangeSinkLocalState:"); + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); auto& p = _parent->cast(); - _dependency = (AnalyticDependency*)info.dependency; - _shared_state = (AnalyticSharedState*)_dependency->shared_state(); _shared_state->partition_by_column_idxs.resize(p._partition_by_eq_expr_ctxs.size()); _shared_state->ordey_by_column_idxs.resize(p._order_by_eq_expr_ctxs.size()); - _profile = state->obj_pool()->add(new RuntimeProfile("AnalyticSinkLocalState")); _memory_usage_counter = ADD_LABEL_COUNTER(profile(), "MemoryUsage"); _blocks_memory_usage = _profile->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage"); _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime"); @@ -73,11 +69,10 @@ AnalyticSinkOperatorX::AnalyticSinkOperatorX(ObjectPool* pool, const TPlanNode& : DataSinkOperatorX(tnode.node_id), _buffered_tuple_id(tnode.analytic_node.__isset.buffered_tuple_id ? tnode.analytic_node.buffered_tuple_id - : 0) { - _name = "SortSinkOperatorX"; -} + : 0) {} Status AnalyticSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(DataSinkOperatorX::init(tnode, state)); const TAnalyticNode& analytic_node = tnode.analytic_node; size_t agg_size = analytic_node.analytic_functions.size(); _agg_expr_ctxs.resize(agg_size); @@ -141,12 +136,6 @@ Status AnalyticSinkOperatorX::open(RuntimeState* state) { return Status::OK(); } -Status AnalyticSinkOperatorX::setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) { - auto local_state = AnalyticSinkLocalState::create_shared(this, state); - state->emplace_sink_local_state(id(), local_state); - return local_state->init(state, info); -} - Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* input_block, SourceState source_state) { auto& local_state = state->get_sink_local_state(id())->cast(); diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index ed15cf3ab0d90ac..14abe5a6d8f69a0 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -22,6 +22,7 @@ #include "operator.h" #include "pipeline/pipeline_x/dependency.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/vanalytic_eval_node.h" namespace doris { @@ -46,19 +47,17 @@ class AnalyticSinkOperator final : public StreamingOperator { ENABLE_FACTORY_CREATOR(AnalyticSinkLocalState); public: - AnalyticSinkLocalState(DataSinkOperatorX* parent, RuntimeState* state) - : PipelineXSinkLocalState(parent, state) {} + AnalyticSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : PipelineXSinkLocalState(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; private: friend class AnalyticSinkOperatorX; - AnalyticDependency* _dependency; - AnalyticSharedState* _shared_state; RuntimeProfile::Counter* _memory_usage_counter; RuntimeProfile::Counter* _evaluation_timer; @@ -67,28 +66,24 @@ class AnalyticSinkLocalState : public PipelineXSinkLocalState { std::vector _agg_expr_ctxs; }; -class AnalyticSinkOperatorX final : public DataSinkOperatorX { +class AnalyticSinkOperatorX final : public DataSinkOperatorX { public: AnalyticSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); Status init(const TDataSink& tsink) override { - return Status::InternalError("{} should not init with TPlanNode", _name); + return Status::InternalError("{} should not init with TPlanNode", + DataSinkOperatorX::_name); } Status init(const TPlanNode& tnode, RuntimeState* state) override; Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; - Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) override; Status sink(RuntimeState* state, vectorized::Block* in_block, SourceState source_state) override; bool can_write(RuntimeState* state) override; - void get_dependency(DependencySPtr& dependency) override { - dependency.reset(new AnalyticDependency(id())); - } - private: Status _insert_range_column(vectorized::Block* block, const vectorized::VExprContextSPtr& expr, vectorized::IColumn* dst_column, size_t length); diff --git a/be/src/pipeline/exec/analytic_source_operator.cpp b/be/src/pipeline/exec/analytic_source_operator.cpp index 49ab6dc2f811b9f..6ae51250581f685 100644 --- a/be/src/pipeline/exec/analytic_source_operator.cpp +++ b/be/src/pipeline/exec/analytic_source_operator.cpp @@ -26,7 +26,7 @@ namespace doris::pipeline { OPERATOR_CODE_GENERATOR(AnalyticSourceOperator, SourceOperator) AnalyticLocalState::AnalyticLocalState(RuntimeState* state, OperatorXBase* parent) - : PipelineXLocalState(state, parent), + : PipelineXLocalState(state, parent), _output_block_index(0), _window_end_position(0), _next_partition(false), @@ -37,10 +37,8 @@ AnalyticLocalState::AnalyticLocalState(RuntimeState* state, OperatorXBase* paren _agg_functions_created(false) {} Status AnalyticLocalState::init(RuntimeState* state, LocalStateInfo& info) { - RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); _agg_arena_pool = std::make_unique(); - _dependency = (AnalyticDependency*)info.dependency; - _shared_state = (AnalyticSharedState*)_dependency->shared_state(); auto& p = _parent->cast(); _agg_functions_size = p._agg_functions.size(); @@ -344,8 +342,8 @@ void AnalyticLocalState::release_mem() { } AnalyticSourceOperatorX::AnalyticSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs, std::string op_name) - : OperatorXBase(pool, tnode, descs, op_name), + const DescriptorTbl& descs) + : OperatorX(pool, tnode, descs), _window(tnode.analytic_node.window), _intermediate_tuple_id(tnode.analytic_node.intermediate_tuple_id), _output_tuple_id(tnode.analytic_node.output_tuple_id), @@ -375,7 +373,7 @@ AnalyticSourceOperatorX::AnalyticSourceOperatorX(ObjectPool* pool, const TPlanNo } Status AnalyticSourceOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { - RETURN_IF_ERROR(OperatorXBase::init(tnode, state)); + RETURN_IF_ERROR(OperatorX::init(tnode, state)); const TAnalyticNode& analytic_node = tnode.analytic_node; size_t agg_size = analytic_node.analytic_functions.size(); @@ -429,12 +427,6 @@ Status AnalyticSourceOperatorX::get_block(RuntimeState* state, vectorized::Block return Status::OK(); } -Status AnalyticSourceOperatorX::setup_local_state(RuntimeState* state, LocalStateInfo& info) { - auto local_state = AnalyticLocalState::create_shared(state, this); - state->emplace_local_state(id(), local_state); - return local_state->init(state, info); -} - bool AnalyticSourceOperatorX::can_read(RuntimeState* state) { auto& local_state = state->get_local_state(id())->cast(); if (local_state._shared_state->need_more_input) { @@ -453,11 +445,11 @@ Status AnalyticLocalState::close(RuntimeState* state) { _destroy_agg_status(); release_mem(); - return PipelineXLocalState::close(state); + return PipelineXLocalState::close(state); } Status AnalyticSourceOperatorX::prepare(RuntimeState* state) { - RETURN_IF_ERROR(OperatorXBase::prepare(state)); + RETURN_IF_ERROR(OperatorX::prepare(state)); DCHECK(_child_x->row_desc().is_prefix_of(_row_descriptor)); _intermediate_tuple_desc = state->desc_tbl().get_tuple_descriptor(_intermediate_tuple_id); _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_output_tuple_id); @@ -494,7 +486,7 @@ Status AnalyticSourceOperatorX::prepare(RuntimeState* state) { } Status AnalyticSourceOperatorX::open(RuntimeState* state) { - RETURN_IF_ERROR(OperatorXBase::open(state)); + RETURN_IF_ERROR(OperatorX::open(state)); for (auto* agg_function : _agg_functions) { RETURN_IF_ERROR(agg_function->open(state)); } diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index 7178e46742ceb1a..866177ff7154f33 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -21,6 +21,7 @@ #include "common/status.h" #include "operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/vanalytic_eval_node.h" namespace doris { @@ -46,7 +47,7 @@ class AnalyticSourceOperator final : public SourceOperator { public: ENABLE_FACTORY_CREATOR(AnalyticLocalState); AnalyticLocalState(RuntimeState* state, OperatorXBase* parent); @@ -79,9 +80,6 @@ class AnalyticLocalState final : public PipelineXLocalState { friend class AnalyticSourceOperatorX; - AnalyticDependency* _dependency; - AnalyticSharedState* _shared_state; - int64_t _output_block_index; int64_t _window_end_position; bool _next_partition; @@ -116,14 +114,11 @@ class AnalyticLocalState final : public PipelineXLocalState { executor _executor; }; -class AnalyticSourceOperatorX final : public OperatorXBase { +class AnalyticSourceOperatorX final : public OperatorX { public: - AnalyticSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string op_name); + AnalyticSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); bool can_read(RuntimeState* state) override; - Status setup_local_state(RuntimeState* state, LocalStateInfo& info) override; - Status get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) override; diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 48050c83cc0e81d..118b2a31b5ed30d 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -48,8 +48,7 @@ namespace doris::pipeline { template ExchangeSinkBuffer::ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, - int send_id, int be_number, - PipelineFragmentContext* context) + int send_id, int be_number, QueryContext* context) : _is_finishing(false), _query_id(query_id), _dest_node_id(dest_node_id), @@ -84,7 +83,7 @@ bool ExchangeSinkBuffer::is_pending_finish() { // 1 make ExchangeSinkBuffer support try close which calls brpc::StartCancel // 2 make BlockScheduler calls tryclose when query is cancel DCHECK(_context != nullptr); - bool need_cancel = _context->is_canceled(); + bool need_cancel = _context->is_cancelled(); for (auto& pair : _instance_to_package_queue_mutex) { std::unique_lock lock(*(pair.second)); @@ -235,8 +234,7 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { { SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); if (enable_http_send_block(*brpc_request)) { - RETURN_IF_ERROR(transmit_block_http(_context->get_runtime_state(), closure, - *brpc_request, + RETURN_IF_ERROR(transmit_block_http(_context->exec_env(), closure, *brpc_request, request.channel->_brpc_dest_addr)); } else { transmit_block(*request.channel->_brpc_stub, closure, *brpc_request); @@ -288,8 +286,7 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { { SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); if (enable_http_send_block(*brpc_request)) { - RETURN_IF_ERROR(transmit_block_http(_context->get_runtime_state(), closure, - *brpc_request, + RETURN_IF_ERROR(transmit_block_http(_context->exec_env(), closure, *brpc_request, request.channel->_brpc_dest_addr)); } else { transmit_block(*request.channel->_brpc_stub, closure, *brpc_request); @@ -326,7 +323,7 @@ void ExchangeSinkBuffer::_ended(InstanceLoId id) { template void ExchangeSinkBuffer::_failed(InstanceLoId id, const std::string& err) { _is_finishing = true; - _context->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, err); + _context->cancel(true, err, Status::Cancelled(err)); _ended(id); } diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index d9a0d316c6ca8c1..f6c702b4178def7 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -99,8 +99,6 @@ struct BroadcastTransmitInfo { bool eos; }; -class PipelineFragmentContext; - template class SelfDeleteClosure : public google::protobuf::Closure { public: @@ -167,7 +165,7 @@ struct ExchangeRpcContext { template class ExchangeSinkBuffer { public: - ExchangeSinkBuffer(PUniqueId, int, PlanNodeId, int, PipelineFragmentContext*); + ExchangeSinkBuffer(PUniqueId, int, PlanNodeId, int, QueryContext*); ~ExchangeSinkBuffer(); void register_sink(TUniqueId); @@ -207,7 +205,7 @@ class ExchangeSinkBuffer { int _sender_id; int _be_number; std::atomic _rpc_count = 0; - PipelineFragmentContext* _context; + QueryContext* _context; Status _send_rpc(InstanceLoId); // must hold the _instance_to_package_queue_mutex[id] mutex to opera diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index 5168916be6638bc..977781e36dd3ad2 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -26,7 +26,6 @@ #include "common/status.h" #include "exchange_sink_buffer.h" #include "pipeline/exec/operator.h" -#include "pipeline/pipeline_x/pipeline_x_fragment_context.h" #include "vec/columns/column_const.h" #include "vec/exprs/vexpr.h" #include "vec/sink/vdata_stream_sender.h" @@ -38,21 +37,16 @@ class DataSink; namespace doris::pipeline { ExchangeSinkOperatorBuilder::ExchangeSinkOperatorBuilder(int32_t id, DataSink* sink, - PipelineFragmentContext* context, int mult_cast_id) - : DataSinkOperatorBuilder(id, "ExchangeSinkOperator", sink), - _context(context), - _mult_cast_id(mult_cast_id) {} + : DataSinkOperatorBuilder(id, "ExchangeSinkOperator", sink), _mult_cast_id(mult_cast_id) {} OperatorPtr ExchangeSinkOperatorBuilder::build_operator() { - return std::make_shared(this, _sink, _context, _mult_cast_id); + return std::make_shared(this, _sink, _mult_cast_id); } ExchangeSinkOperator::ExchangeSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink, - PipelineFragmentContext* context, int mult_cast_id) - : DataSinkOperator(operator_builder, sink), - _context(context), - _mult_cast_id(mult_cast_id) {} + int mult_cast_id) + : DataSinkOperator(operator_builder, sink), _mult_cast_id(mult_cast_id) {} Status ExchangeSinkOperator::init(const TDataSink& tsink) { // -1 means not the mult cast stream sender @@ -70,7 +64,7 @@ Status ExchangeSinkOperator::prepare(RuntimeState* state) { id.set_hi(_state->query_id().hi); id.set_lo(_state->query_id().lo); _sink_buffer = std::make_unique>( - id, _dest_node_id, _sink->_sender_id, _state->be_number(), _context); + id, _dest_node_id, _sink->_sender_id, _state->be_number(), state->get_query_ctx()); RETURN_IF_ERROR(DataSinkOperator::prepare(state)); _sink->registe_channels(_sink_buffer.get()); @@ -102,7 +96,7 @@ bool ExchangeSinkLocalState::transfer_large_data_by_brpc() const { } Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXSinkLocalState<>::init(state, info)); _sender_id = info.sender_id; _broadcast_pb_blocks.resize(config::num_broadcast_buffer); _broadcast_pb_block_idx = 0; @@ -127,7 +121,6 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf instances.emplace_back(channel->get_fragment_instance_id_str()); } std::string title = "VDataStreamSender (dst_id={}, dst_fragments=[{}])"; - _profile = p._pool->add(new RuntimeProfile(title)); SCOPED_TIMER(_profile->total_time_counter()); SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); @@ -154,7 +147,7 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf id.set_hi(_state->query_id().hi); id.set_lo(_state->query_id().lo); _sink_buffer = std::make_unique>( - id, p._dest_node_id, _sender_id, _state->be_number(), p._context); + id, p._dest_node_id, _sender_id, _state->be_number(), state->get_query_ctx()); register_channels(_sink_buffer.get()); @@ -190,9 +183,8 @@ segment_v2::CompressionTypePB& ExchangeSinkLocalState::compression_type() { ExchangeSinkOperatorX::ExchangeSinkOperatorX( RuntimeState* state, ObjectPool* pool, const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, - bool send_query_statistics_with_every_batch, PipelineXFragmentContext* context) + bool send_query_statistics_with_every_batch) : DataSinkOperatorX(sink.dest_node_id), - _context(context), _pool(pool), _row_desc(row_desc), _part_type(sink.output_partition.type), @@ -212,9 +204,8 @@ ExchangeSinkOperatorX::ExchangeSinkOperatorX( ExchangeSinkOperatorX::ExchangeSinkOperatorX( ObjectPool* pool, const RowDescriptor& row_desc, PlanNodeId dest_node_id, const std::vector& destinations, - bool send_query_statistics_with_every_batch, PipelineXFragmentContext* context) + bool send_query_statistics_with_every_batch) : DataSinkOperatorX(dest_node_id), - _context(context), _pool(pool), _row_desc(row_desc), _part_type(TPartitionType::UNPARTITIONED), @@ -226,10 +217,8 @@ ExchangeSinkOperatorX::ExchangeSinkOperatorX( } ExchangeSinkOperatorX::ExchangeSinkOperatorX(ObjectPool* pool, const RowDescriptor& row_desc, - bool send_query_statistics_with_every_batch, - PipelineXFragmentContext* context) + bool send_query_statistics_with_every_batch) : DataSinkOperatorX(0), - _context(context), _pool(pool), _row_desc(row_desc), _send_query_statistics_with_every_batch(send_query_statistics_with_every_batch), @@ -253,12 +242,6 @@ Status ExchangeSinkOperatorX::init(const TDataSink& tsink) { return Status::OK(); } -Status ExchangeSinkOperatorX::setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) { - auto local_state = ExchangeSinkLocalState::create_shared(this, state); - state->emplace_sink_local_state(id(), local_state); - return local_state->init(state, info); -} - Status ExchangeSinkOperatorX::prepare(RuntimeState* state) { _state = state; @@ -560,7 +543,7 @@ Status ExchangeSinkLocalState::close(RuntimeState* state) { } _sink_buffer->update_profile(profile()); _sink_buffer->close(); - return PipelineXSinkLocalState::close(state); + return PipelineXSinkLocalState<>::close(state); } bool ExchangeSinkOperatorX::can_write(RuntimeState* state) { diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index c9f9c2528d03d87..ef2353dd1c1c26e 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -24,6 +24,7 @@ #include "common/status.h" #include "exchange_sink_buffer.h" #include "operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/sink/vdata_stream_sender.h" namespace doris { @@ -32,27 +33,22 @@ class RuntimeState; class TDataSink; namespace pipeline { -class PipelineFragmentContext; -class PipelineXFragmentContext; class ExchangeSinkOperatorBuilder final : public DataSinkOperatorBuilder { public: - ExchangeSinkOperatorBuilder(int32_t id, DataSink* sink, PipelineFragmentContext* context, - int mult_cast_id = -1); + ExchangeSinkOperatorBuilder(int32_t id, DataSink* sink, int mult_cast_id = -1); OperatorPtr build_operator() override; private: - PipelineFragmentContext* _context; int _mult_cast_id = -1; }; // Now local exchange is not supported since VDataStreamRecvr is considered as a pipeline broker. class ExchangeSinkOperator final : public DataSinkOperator { public: - ExchangeSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink, - PipelineFragmentContext* context, int mult_cast_id); + ExchangeSinkOperator(OperatorBuilderBase* operator_builder, DataSink* sink, int mult_cast_id); Status init(const TDataSink& tsink) override; Status prepare(RuntimeState* state) override; @@ -67,16 +63,15 @@ class ExchangeSinkOperator final : public DataSinkOperator> _sink_buffer; int _dest_node_id = -1; RuntimeState* _state = nullptr; - PipelineFragmentContext* _context; int _mult_cast_id = -1; }; -class ExchangeSinkLocalState : public PipelineXSinkLocalState { +class ExchangeSinkLocalState : public PipelineXSinkLocalState<> { ENABLE_FACTORY_CREATOR(ExchangeSinkLocalState); public: - ExchangeSinkLocalState(DataSinkOperatorX* parent, RuntimeState* state) - : PipelineXSinkLocalState(parent, state), + ExchangeSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : PipelineXSinkLocalState<>(parent, state), current_channel_idx(0), only_local_exchange(false), _serializer(this) {} @@ -154,27 +149,23 @@ class ExchangeSinkLocalState : public PipelineXSinkLocalState { vectorized::BlockSerializer _serializer; }; -class ExchangeSinkOperatorX final : public DataSinkOperatorX { +class ExchangeSinkOperatorX final : public DataSinkOperatorX { public: ExchangeSinkOperatorX(RuntimeState* state, ObjectPool* pool, const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, - bool send_query_statistics_with_every_batch, - PipelineXFragmentContext* context); + bool send_query_statistics_with_every_batch); ExchangeSinkOperatorX(ObjectPool* pool, const RowDescriptor& row_desc, PlanNodeId dest_node_id, const std::vector& destinations, - bool send_query_statistics_with_every_batch, - PipelineXFragmentContext* context); + bool send_query_statistics_with_every_batch); ExchangeSinkOperatorX(ObjectPool* pool, const RowDescriptor& row_desc, - bool send_query_statistics_with_every_batch, - PipelineXFragmentContext* context); + bool send_query_statistics_with_every_batch); Status init(const TDataSink& tsink) override; RuntimeState* state() { return _state; } Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; - Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) override; Status sink(RuntimeState* state, vectorized::Block* in_block, SourceState source_state) override; @@ -205,7 +196,6 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX { const uint64_t* channel_ids, int rows, vectorized::Block* block, bool eos); RuntimeState* _state = nullptr; - PipelineXFragmentContext* _context; ObjectPool* _pool; const RowDescriptor& _row_desc; diff --git a/be/src/pipeline/exec/exchange_source_operator.cpp b/be/src/pipeline/exec/exchange_source_operator.cpp index dadf70f0deffab5..587b109d6bc3973 100644 --- a/be/src/pipeline/exec/exchange_source_operator.cpp +++ b/be/src/pipeline/exec/exchange_source_operator.cpp @@ -41,24 +41,19 @@ bool ExchangeSourceOperator::is_pending_finish() const { } ExchangeLocalState::ExchangeLocalState(RuntimeState* state, OperatorXBase* parent) - : PipelineXLocalState(state, parent), num_rows_skipped(0), is_ready(false) {} + : PipelineXLocalState<>(state, parent), num_rows_skipped(0), is_ready(false) {} Status ExchangeLocalState::init(RuntimeState* state, LocalStateInfo& info) { - if (_init) { - return Status::OK(); - } - RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); stream_recvr = info.recvr; RETURN_IF_ERROR(_parent->cast()._vsort_exec_exprs.clone( state, vsort_exec_exprs)); - _init = true; return Status::OK(); } ExchangeSourceOperatorX::ExchangeSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs, std::string op_name, - int num_senders) - : OperatorXBase(pool, tnode, descs, op_name), + const DescriptorTbl& descs, int num_senders) + : OperatorX(pool, tnode, descs), _num_senders(num_senders), _is_merging(tnode.exchange_node.__isset.sort_info), _input_row_desc(descs, tnode.exchange_node.input_row_tuples, @@ -68,7 +63,7 @@ ExchangeSourceOperatorX::ExchangeSourceOperatorX(ObjectPool* pool, const TPlanNo _offset(tnode.exchange_node.__isset.offset ? tnode.exchange_node.offset : 0) {} Status ExchangeSourceOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { - RETURN_IF_ERROR(OperatorXBase::init(tnode, state)); + RETURN_IF_ERROR(OperatorX::init(tnode, state)); if (!_is_merging) { return Status::OK(); } @@ -80,7 +75,7 @@ Status ExchangeSourceOperatorX::init(const TPlanNode& tnode, RuntimeState* state } Status ExchangeSourceOperatorX::prepare(RuntimeState* state) { - RETURN_IF_ERROR(OperatorXBase::prepare(state)); + RETURN_IF_ERROR(OperatorX::prepare(state)); DCHECK_GT(_num_senders, 0); _sub_plan_query_statistics_recvr.reset(new QueryStatisticsRecvr()); @@ -92,7 +87,7 @@ Status ExchangeSourceOperatorX::prepare(RuntimeState* state) { } Status ExchangeSourceOperatorX::open(RuntimeState* state) { - RETURN_IF_ERROR(OperatorXBase::open(state)); + RETURN_IF_ERROR(OperatorX::open(state)); if (_is_merging) { RETURN_IF_ERROR(_vsort_exec_exprs.open(state)); } @@ -143,12 +138,6 @@ Status ExchangeSourceOperatorX::get_block(RuntimeState* state, vectorized::Block return status; } -Status ExchangeSourceOperatorX::setup_local_state(RuntimeState* state, LocalStateInfo& info) { - auto local_state = ExchangeLocalState::create_shared(state, this); - state->emplace_local_state(id(), local_state); - return local_state->init(state, info); -} - bool ExchangeSourceOperatorX::can_read(RuntimeState* state) { return state->get_local_state(id())->cast().stream_recvr->ready_to_read(); } @@ -167,7 +156,7 @@ Status ExchangeLocalState::close(RuntimeState* state) { if (_parent->cast()._is_merging) { vsort_exec_exprs.close(state); } - return PipelineXLocalState::close(state); + return PipelineXLocalState<>::close(state); } Status ExchangeSourceOperatorX::close(RuntimeState* state) { @@ -175,6 +164,6 @@ Status ExchangeSourceOperatorX::close(RuntimeState* state) { _vsort_exec_exprs.close(state); } _is_closed = true; - return OperatorXBase::close(state); + return OperatorX::close(state); } } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/exchange_source_operator.h b/be/src/pipeline/exec/exchange_source_operator.h index d599d3d06dd0d1b..9e52fcd0fc3179b 100644 --- a/be/src/pipeline/exec/exchange_source_operator.h +++ b/be/src/pipeline/exec/exchange_source_operator.h @@ -20,6 +20,7 @@ #include #include "operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/vexchange_node.h" namespace doris { @@ -50,7 +51,7 @@ class ExchangeSourceOperator final : public SourceOperator { ENABLE_FACTORY_CREATOR(ExchangeLocalState); ExchangeLocalState(RuntimeState* state, OperatorXBase* parent); @@ -63,17 +64,16 @@ class ExchangeLocalState : public PipelineXLocalState { bool is_ready; }; -class ExchangeSourceOperatorX final : public OperatorXBase { +class ExchangeSourceOperatorX final : public OperatorX { public: ExchangeSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string op_name, int num_senders); + int num_senders); bool can_read(RuntimeState* state) override; bool is_pending_finish(RuntimeState* state) const override; Status init(const TPlanNode& tnode, RuntimeState* state) override; Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; - Status setup_local_state(RuntimeState* state, LocalStateInfo& info) override; Status get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) override; diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 55aa25397c97b05..ae8974fb42a9cb2 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -19,10 +19,576 @@ #include +#include "exprs/bloom_filter_func.h" +#include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/operator.h" +#include "vec/common/aggregation_common.h" +#include "vec/utils/template_helpers.hpp" namespace doris::pipeline { OPERATOR_CODE_GENERATOR(HashJoinBuildSink, StreamingOperator) -} // namespace doris::pipeline \ No newline at end of file +template +struct Overload : Callables... { + using Callables::operator()...; +}; + +template +Overload(Callables&&... callables) -> Overload; + +HashJoinBuildSinkLocalState::HashJoinBuildSinkLocalState(DataSinkOperatorXBase* parent, + RuntimeState* state) + : JoinBuildSinkLocalState(parent, state), + _build_block_idx(0), + _build_side_mem_used(0), + _build_side_last_mem_used(0) {} + +Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(JoinBuildSinkLocalState::init(state, info)); + auto& p = _parent->cast(); + _shared_state->join_op_variants = p._join_op_variants; + _shared_state->probe_key_sz = p._build_key_sz; + _shared_state->build_blocks.reset(new std::vector()); + // avoid vector expand change block address. + // one block can store 4g data, _build_blocks can store 128*4g data. + // if probe data bigger than 512g, runtime filter maybe will core dump when insert data. + _shared_state->build_blocks->reserve(vectorized::HASH_JOIN_MAX_BUILD_BLOCK_COUNT); + _shared_state->is_null_safe_eq_join = p._is_null_safe_eq_join; + _shared_state->store_null_in_hash_table = p._store_null_in_hash_table; + _build_expr_ctxs.resize(p._build_expr_ctxs.size()); + for (size_t i = 0; i < _build_expr_ctxs.size(); i++) { + RETURN_IF_ERROR(p._build_expr_ctxs[i]->clone(state, _build_expr_ctxs[i])); + } + _shared_state->build_exprs_size = _build_expr_ctxs.size(); + + _should_build_hash_table = true; + if (p._is_broadcast_join) { + profile()->add_info_string("BroadcastJoin", "true"); + if (state->enable_share_hash_table_for_broadcast_join()) { + profile()->add_info_string("ShareHashTableEnabled", "true"); + _should_build_hash_table = p._shared_hashtable_controller->should_build_hash_table( + state->fragment_instance_id(), p.id()); + } else { + profile()->add_info_string("ShareHashTableEnabled", "false"); + } + } + + _memory_usage_counter = ADD_LABEL_COUNTER(profile(), "MemoryUsage"); + + _build_blocks_memory_usage = + ADD_CHILD_COUNTER(profile(), "BuildBlocks", TUnit::BYTES, "MemoryUsage"); + _hash_table_memory_usage = + ADD_CHILD_COUNTER(profile(), "HashTable", TUnit::BYTES, "MemoryUsage"); + _build_arena_memory_usage = + profile()->AddHighWaterMarkCounter("BuildKeyArena", TUnit::BYTES, "MemoryUsage"); + + // Build phase + auto record_profile = _should_build_hash_table ? _build_phase_profile : faker_runtime_profile(); + _build_table_timer = ADD_CHILD_TIMER(_build_phase_profile, "BuildTableTime", "BuildTime"); + _build_side_merge_block_timer = + ADD_CHILD_TIMER(_build_phase_profile, "BuildSideMergeBlockTime", "BuildTime"); + _build_table_insert_timer = ADD_TIMER(record_profile, "BuildTableInsertTime"); + _build_expr_call_timer = ADD_TIMER(record_profile, "BuildExprCallTime"); + _build_table_expanse_timer = ADD_TIMER(record_profile, "BuildTableExpanseTime"); + _build_table_convert_timer = ADD_TIMER(record_profile, "BuildTableConvertToPartitionedTime"); + _build_side_compute_hash_timer = ADD_TIMER(record_profile, "BuildSideHashComputingTime"); + _build_runtime_filter_timer = ADD_TIMER(record_profile, "BuildRuntimeFilterTime"); + + _open_timer = ADD_TIMER(profile(), "OpenTime"); + _allocate_resource_timer = ADD_TIMER(profile(), "AllocateResourceTime"); + + _build_buckets_counter = ADD_COUNTER(profile(), "BuildBuckets", TUnit::UNIT); + _build_buckets_fill_counter = ADD_COUNTER(profile(), "FilledBuckets", TUnit::UNIT); + + _build_collisions_counter = ADD_COUNTER(profile(), "BuildCollisions", TUnit::UNIT); + // Hash Table Init + _hash_table_init(state); + + _runtime_filters.resize(p._runtime_filter_descs.size()); + for (size_t i = 0; i < p._runtime_filter_descs.size(); i++) { + RETURN_IF_ERROR(state->runtime_filter_mgr()->register_producer_filter( + p._runtime_filter_descs[i], state->query_options(), _build_expr_ctxs.size() == 1)); + RETURN_IF_ERROR(state->runtime_filter_mgr()->get_producer_filter( + p._runtime_filter_descs[i].filter_id, &_runtime_filters[i])); + } + + for (size_t i = 0; i < p._runtime_filter_descs.size(); i++) { + if (auto bf = _runtime_filters[i]->get_bloomfilter()) { + RETURN_IF_ERROR(bf->init_with_fixed_length()); + } + } + return Status::OK(); +} + +void HashJoinBuildSinkLocalState::init_short_circuit_for_probe() { + auto& p = _parent->cast(); + _shared_state->short_circuit_for_probe = + (_short_circuit_for_null_in_probe_side && + p._join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) || + (_shared_state->build_blocks->empty() && p._join_op == TJoinOp::INNER_JOIN && + !p._is_mark_join) || + (_shared_state->build_blocks->empty() && p._join_op == TJoinOp::LEFT_SEMI_JOIN && + !p._is_mark_join) || + (_shared_state->build_blocks->empty() && p._join_op == TJoinOp::RIGHT_OUTER_JOIN) || + (_shared_state->build_blocks->empty() && p._join_op == TJoinOp::RIGHT_SEMI_JOIN) || + (_shared_state->build_blocks->empty() && p._join_op == TJoinOp::RIGHT_ANTI_JOIN); +} + +Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, + vectorized::Block& block, uint8_t offset) { + auto& p = _parent->cast(); + SCOPED_TIMER(_build_table_timer); + size_t rows = block.rows(); + if (UNLIKELY(rows == 0)) { + return Status::OK(); + } + COUNTER_UPDATE(_build_rows_counter, rows); + + vectorized::ColumnRawPtrs raw_ptrs(_build_expr_ctxs.size()); + + vectorized::ColumnUInt8::MutablePtr null_map_val; + std::vector res_col_ids(_build_expr_ctxs.size()); + RETURN_IF_ERROR(_dependency->do_evaluate(block, _build_expr_ctxs, *_build_expr_call_timer, + res_col_ids)); + if (p._join_op == TJoinOp::LEFT_OUTER_JOIN || p._join_op == TJoinOp::FULL_OUTER_JOIN) { + _dependency->convert_block_to_null(block); + } + // TODO: Now we are not sure whether a column is nullable only by ExecNode's `row_desc` + // so we have to initialize this flag by the first build block. + if (!_has_set_need_null_map_for_build) { + _has_set_need_null_map_for_build = true; + _set_build_ignore_flag(block, res_col_ids); + } + if (p._short_circuit_for_null_in_build_side || _build_side_ignore_null) { + null_map_val = vectorized::ColumnUInt8::create(); + null_map_val->get_data().assign(rows, (uint8_t)0); + } + + // Get the key column that needs to be built + Status st = _dependency->extract_join_column(block, null_map_val, raw_ptrs, res_col_ids); + + st = std::visit( + Overload { + [&](std::monostate& arg, auto has_null_value, + auto short_circuit_for_null_in_build_side) -> Status { + LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); + return Status::OK(); + }, + [&](auto&& arg, auto has_null_value, + auto short_circuit_for_null_in_build_side) -> Status { + using HashTableCtxType = std::decay_t; + vectorized::HashJoinBuildContext context(this); + vectorized::ProcessHashTableBuild + hash_table_build_process(rows, block, raw_ptrs, &context, + state->batch_size(), offset, state); + return hash_table_build_process + .template run( + arg, + has_null_value || short_circuit_for_null_in_build_side + ? &null_map_val->get_data() + : nullptr, + &_short_circuit_for_null_in_probe_side); + }}, + *_shared_state->hash_table_variants, + vectorized::make_bool_variant(_build_side_ignore_null), + vectorized::make_bool_variant(p._short_circuit_for_null_in_build_side)); + + return st; +} + +void HashJoinBuildSinkLocalState::_set_build_ignore_flag(vectorized::Block& block, + const std::vector& res_col_ids) { + for (size_t i = 0; i < _build_expr_ctxs.size(); ++i) { + if (!_shared_state->is_null_safe_eq_join[i]) { + auto column = block.get_by_position(res_col_ids[i]).column.get(); + if (check_and_get_column(*column)) { + _build_side_ignore_null |= (_parent->cast()._join_op != + TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN && + !_shared_state->store_null_in_hash_table[i]); + } + } + } +} + +void HashJoinBuildSinkLocalState::_hash_table_init(RuntimeState* state) { + auto& p = _parent->cast(); + std::visit( + [&](auto&& join_op_variants, auto have_other_join_conjunct) { + using JoinOpType = std::decay_t; + using RowRefListType = std::conditional_t< + have_other_join_conjunct, vectorized::RowRefListWithFlags, + std::conditional_t>; + _shared_state->probe_row_match_iter + .emplace>(); + _shared_state->outer_join_pull_visited_iter + .emplace>(); + + if (_build_expr_ctxs.size() == 1 && !p._store_null_in_hash_table[0]) { + // Single column optimization + switch (_build_expr_ctxs[0]->root()->result_type()) { + case TYPE_BOOLEAN: + case TYPE_TINYINT: + _shared_state->hash_table_variants + ->emplace>(); + break; + case TYPE_SMALLINT: + _shared_state->hash_table_variants + ->emplace>(); + break; + case TYPE_INT: + case TYPE_FLOAT: + case TYPE_DATEV2: + _shared_state->hash_table_variants + ->emplace>(); + break; + case TYPE_BIGINT: + case TYPE_DOUBLE: + case TYPE_DATETIME: + case TYPE_DATE: + case TYPE_DATETIMEV2: + _shared_state->hash_table_variants + ->emplace>(); + break; + case TYPE_LARGEINT: + case TYPE_DECIMALV2: + case TYPE_DECIMAL32: + case TYPE_DECIMAL64: + case TYPE_DECIMAL128I: { + vectorized::DataTypePtr& type_ptr = + _build_expr_ctxs[0]->root()->data_type(); + vectorized::TypeIndex idx = + _build_expr_ctxs[0]->root()->is_nullable() + ? assert_cast( + *type_ptr) + .get_nested_type() + ->get_type_id() + : type_ptr->get_type_id(); + vectorized::WhichDataType which(idx); + if (which.is_decimal32()) { + _shared_state->hash_table_variants + ->emplace>(); + } else if (which.is_decimal64()) { + _shared_state->hash_table_variants + ->emplace>(); + } else { + _shared_state->hash_table_variants + ->emplace>(); + } + break; + } + default: + _shared_state->hash_table_variants + ->emplace>(); + } + return; + } + + bool use_fixed_key = true; + bool has_null = false; + size_t key_byte_size = 0; + size_t bitmap_size = vectorized::get_bitmap_size(_build_expr_ctxs.size()); + + for (int i = 0; i < _build_expr_ctxs.size(); ++i) { + const auto vexpr = _build_expr_ctxs[i]->root(); + const auto& data_type = vexpr->data_type(); + + if (!data_type->have_maximum_size_of_value()) { + use_fixed_key = false; + break; + } + + auto is_null = data_type->is_nullable(); + has_null |= is_null; + key_byte_size += p._build_key_sz[i]; + } + + if (bitmap_size + key_byte_size > sizeof(vectorized::UInt256)) { + use_fixed_key = false; + } + + if (use_fixed_key) { + // TODO: may we should support uint256 in the future + if (has_null) { + if (bitmap_size + key_byte_size <= sizeof(vectorized::UInt64)) { + _shared_state->hash_table_variants + ->emplace>(); + } else if (bitmap_size + key_byte_size <= sizeof(vectorized::UInt128)) { + _shared_state->hash_table_variants + ->emplace>(); + } else { + _shared_state->hash_table_variants + ->emplace>(); + } + } else { + if (key_byte_size <= sizeof(vectorized::UInt64)) { + _shared_state->hash_table_variants + ->emplace>(); + } else if (key_byte_size <= sizeof(vectorized::UInt128)) { + _shared_state->hash_table_variants + ->emplace>(); + } else { + _shared_state->hash_table_variants + ->emplace>(); + } + } + } else { + _shared_state->hash_table_variants + ->emplace>(); + } + }, + _shared_state->join_op_variants, + vectorized::make_bool_variant(p._have_other_join_conjunct)); + + DCHECK(!std::holds_alternative(*_shared_state->hash_table_variants)); + + std::visit(vectorized::Overload {[&](std::monostate& arg) { + LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); + }, + [&](auto&& arg) { + arg.hash_table.set_partitioned_threshold( + state->partitioned_hash_join_rows_threshold()); + }}, + *_shared_state->hash_table_variants); +} + +HashJoinBuildSinkOperatorX::HashJoinBuildSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, + const DescriptorTbl& descs) + : JoinBuildSinkOperatorX(pool, tnode, descs), + _is_broadcast_join(tnode.hash_join_node.__isset.is_broadcast_join && + tnode.hash_join_node.is_broadcast_join) { + _runtime_filter_descs = tnode.runtime_filters; +} + +Status HashJoinBuildSinkOperatorX::prepare(RuntimeState* state) { + if (_is_broadcast_join) { + if (state->enable_share_hash_table_for_broadcast_join()) { + _shared_hashtable_controller = + state->get_query_ctx()->get_shared_hash_table_controller(); + _shared_hash_table_context = _shared_hashtable_controller->get_context(id()); + } + } + RETURN_IF_ERROR(vectorized::VExpr::prepare(_build_expr_ctxs, state, _child_x->row_desc())); + return Status::OK(); +} + +Status HashJoinBuildSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(JoinBuildSinkOperatorX::init(tnode, state)); + DCHECK(tnode.__isset.hash_join_node); + + const bool build_stores_null = _join_op == TJoinOp::RIGHT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN || + _join_op == TJoinOp::RIGHT_ANTI_JOIN; + + const std::vector& eq_join_conjuncts = tnode.hash_join_node.eq_join_conjuncts; + for (const auto& eq_join_conjunct : eq_join_conjuncts) { + vectorized::VExprContextSPtr ctx; + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(eq_join_conjunct.right, ctx)); + _build_expr_ctxs.push_back(ctx); + + const auto vexpr = _build_expr_ctxs.back()->root(); + const auto& data_type = vexpr->data_type(); + + if (!data_type->have_maximum_size_of_value()) { + break; + } + + auto is_null = data_type->is_nullable(); + _build_key_sz.push_back(data_type->get_maximum_size_of_value_in_memory() - + (is_null ? 1 : 0)); + + bool null_aware = eq_join_conjunct.__isset.opcode && + eq_join_conjunct.opcode == TExprOpcode::EQ_FOR_NULL; + + _is_null_safe_eq_join.push_back(null_aware); + + // if is null aware, build join column and probe join column both need dispose null value + _store_null_in_hash_table.emplace_back( + null_aware || + (_build_expr_ctxs.back()->root()->is_nullable() && build_stores_null)); + } + + return Status::OK(); +} + +Status HashJoinBuildSinkOperatorX::open(RuntimeState* state) { + return vectorized::VExpr::open(_build_expr_ctxs, state); +} + +Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* in_block, + SourceState source_state) { + auto& local_state = state->get_sink_local_state(id())->cast(); + SCOPED_TIMER(local_state._build_timer); + + // make one block for each 4 gigabytes + constexpr static auto BUILD_BLOCK_MAX_SIZE = 4 * 1024UL * 1024UL * 1024UL; + + if (local_state._short_circuit_for_null_in_probe_side) { + // TODO: if _short_circuit_for_null_in_probe_side is true we should finish current pipeline task. + DCHECK(state->enable_pipeline_exec()); + return Status::OK(); + } + if (local_state._should_build_hash_table) { + // If eos or have already met a null value using short-circuit strategy, we do not need to pull + // data from probe side. + local_state._build_side_mem_used += in_block->allocated_bytes(); + + if (in_block->rows() != 0) { + SCOPED_TIMER(local_state._build_side_merge_block_timer); + RETURN_IF_ERROR(local_state._build_side_mutable_block.merge(*in_block)); + } + + if (UNLIKELY(local_state._build_side_mem_used - local_state._build_side_last_mem_used > + BUILD_BLOCK_MAX_SIZE)) { + if (local_state._shared_state->build_blocks->size() == + vectorized::HASH_JOIN_MAX_BUILD_BLOCK_COUNT) { + return Status::NotSupported(strings::Substitute( + "data size of right table in hash join > $0", + BUILD_BLOCK_MAX_SIZE * vectorized::HASH_JOIN_MAX_BUILD_BLOCK_COUNT)); + } + local_state._shared_state->build_blocks->emplace_back( + local_state._build_side_mutable_block.to_block()); + + COUNTER_UPDATE(local_state._build_blocks_memory_usage, + (*local_state._shared_state->build_blocks)[local_state._build_block_idx] + .bytes()); + + // TODO:: Rethink may we should do the process after we receive all build blocks ? + // which is better. + RETURN_IF_ERROR(local_state.process_build_block( + state, (*local_state._shared_state->build_blocks)[local_state._build_block_idx], + local_state._build_block_idx)); + + local_state._build_side_mutable_block = vectorized::MutableBlock(); + ++local_state._build_block_idx; + local_state._build_side_last_mem_used = local_state._build_side_mem_used; + } + } + + if (local_state._should_build_hash_table && source_state == SourceState::FINISHED) { + if (!local_state._build_side_mutable_block.empty()) { + if (local_state._shared_state->build_blocks->size() == + vectorized::HASH_JOIN_MAX_BUILD_BLOCK_COUNT) { + return Status::NotSupported(strings::Substitute( + "data size of right table in hash join > $0", + BUILD_BLOCK_MAX_SIZE * vectorized::HASH_JOIN_MAX_BUILD_BLOCK_COUNT)); + } + local_state._shared_state->build_blocks->emplace_back( + local_state._build_side_mutable_block.to_block()); + COUNTER_UPDATE(local_state._build_blocks_memory_usage, + (*local_state._shared_state->build_blocks)[local_state._build_block_idx] + .bytes()); + RETURN_IF_ERROR(local_state.process_build_block( + state, (*local_state._shared_state->build_blocks)[local_state._build_block_idx], + local_state._build_block_idx)); + } + auto ret = std::visit(Overload {[&](std::monostate&) -> Status { + LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); + }, + [&](auto&& arg) -> Status { + using HashTableCtxType = std::decay_t; + vectorized::RuntimeFilterContext context(&local_state); + vectorized::ProcessRuntimeFilterBuild + runtime_filter_build_process(&context); + return runtime_filter_build_process(state, arg); + }}, + *local_state._shared_state->hash_table_variants); + if (!ret.ok()) { + if (_shared_hashtable_controller) { + _shared_hash_table_context->status = ret; + _shared_hashtable_controller->signal(id()); + } + return ret; + } + if (_shared_hashtable_controller) { + _shared_hash_table_context->status = Status::OK(); + // arena will be shared with other instances. + _shared_hash_table_context->arena = local_state._shared_state->arena; + _shared_hash_table_context->blocks = local_state._shared_state->build_blocks; + _shared_hash_table_context->hash_table_variants = + local_state._shared_state->hash_table_variants; + _shared_hash_table_context->short_circuit_for_null_in_probe_side = + local_state._short_circuit_for_null_in_probe_side; + if (local_state._runtime_filter_slots) { + local_state._runtime_filter_slots->copy_to_shared_context( + _shared_hash_table_context); + } + _shared_hashtable_controller->signal(id()); + } + } else if (!local_state._should_build_hash_table) { + DCHECK(_shared_hashtable_controller != nullptr); + DCHECK(_shared_hash_table_context != nullptr); + auto wait_timer = ADD_CHILD_TIMER(local_state._build_phase_profile, + "WaitForSharedHashTableTime", "BuildTime"); + SCOPED_TIMER(wait_timer); + RETURN_IF_ERROR( + _shared_hashtable_controller->wait_for_signal(state, _shared_hash_table_context)); + + local_state._build_phase_profile->add_info_string( + "SharedHashTableFrom", + print_id(_shared_hashtable_controller->get_builder_fragment_instance_id(id()))); + local_state._short_circuit_for_null_in_probe_side = + _shared_hash_table_context->short_circuit_for_null_in_probe_side; + local_state._shared_state->hash_table_variants = + std::static_pointer_cast( + _shared_hash_table_context->hash_table_variants); + local_state._shared_state->build_blocks = _shared_hash_table_context->blocks; + + if (!_shared_hash_table_context->runtime_filters.empty()) { + auto ret = std::visit( + Overload { + [&](std::monostate&) -> Status { + LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); + }, + [&](auto&& arg) -> Status { + if (_runtime_filter_descs.empty()) { + return Status::OK(); + } + local_state._runtime_filter_slots = + std::make_shared( + _build_expr_ctxs, _runtime_filter_descs); + + RETURN_IF_ERROR(local_state._runtime_filter_slots->init( + state, arg.hash_table.get_size(), 0)); + RETURN_IF_ERROR( + local_state._runtime_filter_slots->copy_from_shared_context( + _shared_hash_table_context)); + RETURN_IF_ERROR(local_state._runtime_filter_slots->publish()); + return Status::OK(); + }}, + *local_state._shared_state->hash_table_variants); + RETURN_IF_ERROR(ret); + } + } + + local_state.init_short_circuit_for_probe(); + if (source_state == SourceState::FINISHED) { + local_state._dependency->set_done(); + } + + return Status::OK(); +} + +Status HashJoinBuildSinkOperatorX::close(RuntimeState* state) { + if (!is_closed()) { + _shared_hash_table_context = nullptr; + _is_closed = true; + } + return JoinBuildSinkOperatorX::close(state); +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index c1e67512ddd7a61..56d9ec050cdd4ac 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -19,7 +19,9 @@ #include +#include "join_build_sink_operator.h" #include "operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/join/vhash_join_node.h" namespace doris { @@ -42,5 +44,112 @@ class HashJoinBuildSink final : public StreamingOperatorready_for_finish(); } }; +class HashJoinBuildSinkOperatorX; + +class HashJoinBuildSinkLocalState final + : public JoinBuildSinkLocalState { +public: + ENABLE_FACTORY_CREATOR(HashJoinBuildSinkLocalState); + using Parent = HashJoinBuildSinkOperatorX; + HashJoinBuildSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); + ~HashJoinBuildSinkLocalState() = default; + + Status init(RuntimeState* state, LocalSinkStateInfo& info) override; + Status process_build_block(RuntimeState* state, vectorized::Block& block, uint8_t offset); + + void init_short_circuit_for_probe(); + HashJoinBuildSinkOperatorX* join_build() { return (HashJoinBuildSinkOperatorX*)_parent; } + +protected: + void _hash_table_init(RuntimeState* state); + void _set_build_ignore_flag(vectorized::Block& block, const std::vector& res_col_ids); + friend class HashJoinBuildSinkOperatorX; + friend struct vectorized::HashJoinBuildContext; + friend struct vectorized::RuntimeFilterContext; + + // build expr + vectorized::VExprContextSPtrs _build_expr_ctxs; + + std::vector _runtime_filters; + bool _should_build_hash_table = true; + uint8_t _build_block_idx = 0; + int64_t _build_side_mem_used = 0; + int64_t _build_side_last_mem_used = 0; + vectorized::MutableBlock _build_side_mutable_block; + std::shared_ptr _runtime_filter_slots = nullptr; + bool _has_set_need_null_map_for_build = false; + bool _build_side_ignore_null = false; + size_t _build_bf_cardinality = 0; + std::unordered_map> _inserted_rows; + + RuntimeProfile::Counter* _build_table_timer; + RuntimeProfile::Counter* _build_expr_call_timer; + RuntimeProfile::Counter* _build_table_insert_timer; + RuntimeProfile::Counter* _build_table_expanse_timer; + RuntimeProfile::Counter* _build_table_convert_timer; + RuntimeProfile::Counter* _build_buckets_counter; + RuntimeProfile::Counter* _build_buckets_fill_counter; + + RuntimeProfile::Counter* _build_side_compute_hash_timer; + RuntimeProfile::Counter* _build_side_merge_block_timer; + RuntimeProfile::Counter* _build_runtime_filter_timer; + + RuntimeProfile::Counter* _build_collisions_counter; + + RuntimeProfile::Counter* _open_timer; + RuntimeProfile::Counter* _allocate_resource_timer; + + RuntimeProfile::Counter* _memory_usage_counter; + RuntimeProfile::Counter* _build_blocks_memory_usage; + RuntimeProfile::Counter* _hash_table_memory_usage; + RuntimeProfile::HighWaterMarkCounter* _build_arena_memory_usage; +}; + +class HashJoinBuildSinkOperatorX final + : public JoinBuildSinkOperatorX { +public: + HashJoinBuildSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, + const DescriptorTbl& descs); + Status init(const TDataSink& tsink) override { + return Status::InternalError("{} should not init with TDataSink", + JoinBuildSinkOperatorX::_name); + } + + Status init(const TPlanNode& tnode, RuntimeState* state) override; + + Status prepare(RuntimeState* state) override; + Status open(RuntimeState* state) override; + + Status sink(RuntimeState* state, vectorized::Block* in_block, + SourceState source_state) override; + Status close(RuntimeState* state) override; + + virtual bool can_write(RuntimeState* state) override { return true; } + +private: + friend class HashJoinBuildSinkLocalState; + friend struct vectorized::HashJoinBuildContext; + friend struct vectorized::RuntimeFilterContext; + + // build expr + vectorized::VExprContextSPtrs _build_expr_ctxs; + // mark the build hash table whether it needs to store null value + std::vector _store_null_in_hash_table; + + // mark the join column whether support null eq + std::vector _is_null_safe_eq_join; + + vectorized::Sizes _build_key_sz; + + bool _is_broadcast_join = false; + std::shared_ptr _shared_hashtable_controller = nullptr; + + vectorized::SharedHashTableContextPtr _shared_hash_table_context = nullptr; + std::vector _runtime_filter_descs; + + std::atomic_bool _probe_open_finish = false; + bool _probe_ignore_null = false; +}; + } // namespace pipeline -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index e499d391ced3f21..587631ded1e2be3 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -26,5 +26,463 @@ namespace pipeline { OPERATOR_CODE_GENERATOR(HashJoinProbeOperator, StatefulOperator) +HashJoinProbeLocalState::HashJoinProbeLocalState(RuntimeState* state, OperatorXBase* parent) + : JoinProbeLocalState(state, parent), + _child_block(vectorized::Block::create_unique()), + _child_source_state(SourceState::DEPEND_ON_SOURCE) {} + +Status HashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(JoinProbeLocalState::init(state, info)); + auto& p = _parent->cast(); + _probe_ignore_null = p._probe_ignore_null; + _probe_expr_ctxs.resize(p._probe_expr_ctxs.size()); + for (size_t i = 0; i < _probe_expr_ctxs.size(); i++) { + RETURN_IF_ERROR(p._probe_expr_ctxs[i]->clone(state, _probe_expr_ctxs[i])); + } + _other_join_conjuncts.resize(p._other_join_conjuncts.size()); + for (size_t i = 0; i < _other_join_conjuncts.size(); i++) { + RETURN_IF_ERROR(p._other_join_conjuncts[i]->clone(state, _other_join_conjuncts[i])); + } + // Since the comparison of null values is meaningless, null aware left anti join should not output null + // when the build side is not empty. + if (!_shared_state->build_blocks->empty() && p._join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { + _probe_ignore_null = true; + } + _construct_mutable_join_block(); + _probe_column_disguise_null.reserve(_probe_expr_ctxs.size()); + _probe_arena_memory_usage = + profile()->AddHighWaterMarkCounter("ProbeKeyArena", TUnit::BYTES, "MemoryUsage"); + // Probe phase + auto probe_phase_profile = _probe_phase_profile; + _probe_next_timer = ADD_TIMER(probe_phase_profile, "ProbeFindNextTime"); + _probe_expr_call_timer = ADD_TIMER(probe_phase_profile, "ProbeExprCallTime"); + _search_hashtable_timer = + ADD_CHILD_TIMER(probe_phase_profile, "ProbeWhenSearchHashTableTime", "ProbeTime"); + _build_side_output_timer = + ADD_CHILD_TIMER(probe_phase_profile, "ProbeWhenBuildSideOutputTime", "ProbeTime"); + _probe_side_output_timer = + ADD_CHILD_TIMER(probe_phase_profile, "ProbeWhenProbeSideOutputTime", "ProbeTime"); + _probe_process_hashtable_timer = + ADD_CHILD_TIMER(probe_phase_profile, "ProbeWhenProcessHashTableTime", "ProbeTime"); + _process_other_join_conjunct_timer = ADD_TIMER(profile(), "OtherJoinConjunctTime"); + return Status::OK(); +} + +void HashJoinProbeLocalState::prepare_for_next() { + _probe_index = 0; + _prepare_probe_block(); +} + +Status HashJoinProbeLocalState::close(RuntimeState* state) { + if (_closed) { + return Status::OK(); + } + std::visit(vectorized::Overload {[&](std::monostate&) {}, + [&](auto&& process_hashtable_ctx) { + if (process_hashtable_ctx._arena) { + process_hashtable_ctx._arena.reset(); + } + + if (process_hashtable_ctx._serialize_key_arena) { + process_hashtable_ctx._serialize_key_arena.reset(); + process_hashtable_ctx._serialized_key_buffer_size = 0; + } + }}, + *_process_hashtable_ctx_variants); + _shared_state->arena = nullptr; + _shared_state->hash_table_variants.reset(); + _process_hashtable_ctx_variants = nullptr; + _null_map_column = nullptr; + _tuple_is_null_left_flag_column = nullptr; + _tuple_is_null_right_flag_column = nullptr; + _probe_block.clear(); + return JoinProbeLocalState::close(state); +} + +bool HashJoinProbeLocalState::_need_probe_null_map(vectorized::Block& block, + const std::vector& res_col_ids) { + for (size_t i = 0; i < _probe_expr_ctxs.size(); ++i) { + if (!_shared_state->is_null_safe_eq_join[i]) { + auto column = block.get_by_position(res_col_ids[i]).column.get(); + if (check_and_get_column(*column)) { + return true; + } + } + } + return false; +} + +void HashJoinProbeLocalState::init_for_probe(RuntimeState* state) { + if (_probe_inited) { + return; + } + _process_hashtable_ctx_variants = std::make_unique(); + std::visit( + [&](auto&& join_op_variants) { + using JoinOpType = std::decay_t; + _probe_context.reset(new vectorized::HashJoinProbeContext(this)); + _process_hashtable_ctx_variants + ->emplace>( + _probe_context.get(), state->batch_size()); + }, + _shared_state->join_op_variants); + _probe_inited = true; +} + +void HashJoinProbeLocalState::add_tuple_is_null_column(vectorized::Block* block) { + DCHECK(_parent->cast()._is_outer_join); + auto p0 = _tuple_is_null_left_flag_column->assume_mutable(); + auto p1 = _tuple_is_null_right_flag_column->assume_mutable(); + auto& left_null_map = reinterpret_cast(*p0); + auto& right_null_map = reinterpret_cast(*p1); + auto left_size = left_null_map.size(); + auto right_size = right_null_map.size(); + + if (left_size == 0) { + DCHECK_EQ(right_size, block->rows()); + left_null_map.get_data().resize_fill(right_size, 0); + } + if (right_size == 0) { + DCHECK_EQ(left_size, block->rows()); + right_null_map.get_data().resize_fill(left_size, 0); + } + + block->insert( + {std::move(p0), std::make_shared(), "left_tuples_is_null"}); + block->insert( + {std::move(p1), std::make_shared(), "right_tuples_is_null"}); +} + +void HashJoinProbeLocalState::_prepare_probe_block() { + // clear_column_data of _probe_block + if (!_probe_column_disguise_null.empty()) { + for (int i = 0; i < _probe_column_disguise_null.size(); ++i) { + auto column_to_erase = _probe_column_disguise_null[i]; + _probe_block.erase(column_to_erase - i); + } + _probe_column_disguise_null.clear(); + } + + // remove add nullmap of probe columns + for (auto index : _probe_column_convert_to_null) { + auto& column_type = _probe_block.safe_get_by_position(index); + DCHECK(column_type.column->is_nullable() || is_column_const(*(column_type.column.get()))); + DCHECK(column_type.type->is_nullable()); + + column_type.column = remove_nullable(column_type.column); + column_type.type = remove_nullable(column_type.type); + } + _probe_block.clear_column_data(_parent->get_child()->row_desc().num_materialized_slots()); +} + +HashJoinProbeOperatorX::HashJoinProbeOperatorX(ObjectPool* pool, const TPlanNode& tnode, + const DescriptorTbl& descs) + : JoinProbeOperatorX(pool, tnode, descs), + _hash_output_slot_ids(tnode.hash_join_node.__isset.hash_output_slot_ids + ? tnode.hash_join_node.hash_output_slot_ids + : std::vector {}) {} + +Status HashJoinProbeOperatorX::get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) { + auto& local_state = state->get_local_state(id())->cast(); + local_state.init_for_probe(state); + if (need_more_input_data(state)) { + local_state._child_block->clear_column_data(); + RETURN_IF_ERROR(_child_x->get_block(state, local_state._child_block.get(), + local_state._child_source_state)); + source_state = local_state._child_source_state; + if (local_state._child_block->rows() == 0 && + local_state._child_source_state != SourceState::FINISHED) { + return Status::OK(); + } + local_state.prepare_for_next(); + RETURN_IF_ERROR( + push(state, local_state._child_block.get(), local_state._child_source_state)); + } + + if (!need_more_input_data(state)) { + RETURN_IF_ERROR(pull(state, block, source_state)); + if (source_state != SourceState::FINISHED && !need_more_input_data(state)) { + source_state = SourceState::MORE_DATA; + } else if (source_state != SourceState::FINISHED && + source_state == SourceState::MORE_DATA) { + source_state = local_state._child_source_state; + } + } + return Status::OK(); +} + +Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Block* output_block, + SourceState& source_state) { + auto& local_state = state->get_local_state(id())->cast(); + SCOPED_TIMER(local_state._probe_timer); + if (local_state._shared_state->short_circuit_for_probe) { + // If we use a short-circuit strategy, should return empty block directly. + source_state = SourceState::FINISHED; + return Status::OK(); + } + local_state._join_block.clear_column_data(); + + vectorized::MutableBlock mutable_join_block(&local_state._join_block); + vectorized::Block temp_block; + + Status st; + if (local_state._probe_index < local_state._probe_block.rows()) { + DCHECK(local_state._has_set_need_null_map_for_probe); + RETURN_IF_CATCH_EXCEPTION({ + std::visit( + [&](auto&& arg, auto&& process_hashtable_ctx, auto need_null_map_for_probe, + auto ignore_null) { + using HashTableProbeType = std::decay_t; + if constexpr (!std::is_same_v) { + using HashTableCtxType = std::decay_t; + if constexpr (!std::is_same_v) { + if (_have_other_join_conjunct) { + st = process_hashtable_ctx + .template do_process_with_other_join_conjuncts< + need_null_map_for_probe, ignore_null>( + arg, + need_null_map_for_probe + ? &local_state._null_map_column + ->get_data() + : nullptr, + mutable_join_block, &temp_block, + local_state._probe_block.rows(), + _is_mark_join); + } else { + st = process_hashtable_ctx.template do_process< + need_null_map_for_probe, ignore_null>( + arg, + need_null_map_for_probe + ? &local_state._null_map_column->get_data() + : nullptr, + mutable_join_block, &temp_block, + local_state._probe_block.rows(), _is_mark_join); + } + } else { + LOG(FATAL) << "FATAL: uninited hash table"; + } + } else { + LOG(FATAL) << "FATAL: uninited hash table probe"; + } + }, + *local_state._shared_state->hash_table_variants, + *local_state._process_hashtable_ctx_variants, + vectorized::make_bool_variant(local_state._need_null_map_for_probe), + vectorized::make_bool_variant(local_state._probe_ignore_null)); + }); + } else if (local_state._probe_eos) { + if (_is_right_semi_anti || (_is_outer_join && _join_op != TJoinOp::LEFT_OUTER_JOIN)) { + std::visit( + [&](auto&& arg, auto&& process_hashtable_ctx) { + using HashTableProbeType = std::decay_t; + if constexpr (!std::is_same_v) { + using HashTableCtxType = std::decay_t; + if constexpr (!std::is_same_v) { + bool eos; + st = process_hashtable_ctx.process_data_in_hashtable( + arg, mutable_join_block, &temp_block, &eos); + source_state = eos ? SourceState::FINISHED : source_state; + } else { + LOG(FATAL) << "FATAL: uninited hash table"; + } + } else { + LOG(FATAL) << "FATAL: uninited hash table probe"; + } + }, + *local_state._shared_state->hash_table_variants, + *local_state._process_hashtable_ctx_variants); + } else { + source_state = SourceState::FINISHED; + return Status::OK(); + } + } else { + return Status::OK(); + } + if (!st) { + return st; + } + if (_is_outer_join) { + local_state.add_tuple_is_null_column(&temp_block); + } + auto output_rows = temp_block.rows(); + DCHECK(output_rows <= state->batch_size()); + { + SCOPED_TIMER(local_state._join_filter_timer); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, &temp_block, + temp_block.columns())); + } + + // Here make _join_block release the columns' ptr + local_state._join_block.set_columns(local_state._join_block.clone_empty_columns()); + mutable_join_block.clear(); + + RETURN_IF_ERROR(local_state._build_output_block(&temp_block, output_block, false)); + local_state._reset_tuple_is_null_column(); + local_state.reached_limit(output_block, source_state); + return Status::OK(); +} + +bool HashJoinProbeOperatorX::need_more_input_data(RuntimeState* state) const { + auto& local_state = state->get_local_state(id())->cast(); + return (local_state._probe_block.rows() == 0 || + local_state._probe_index == local_state._probe_block.rows()) && + !local_state._probe_eos && !local_state._shared_state->short_circuit_for_probe; +} + +Status HashJoinProbeOperatorX::_do_evaluate(vectorized::Block& block, + vectorized::VExprContextSPtrs& exprs, + RuntimeProfile::Counter& expr_call_timer, + std::vector& res_col_ids) { + for (size_t i = 0; i < exprs.size(); ++i) { + int result_col_id = -1; + // execute build column + { + SCOPED_TIMER(&expr_call_timer); + RETURN_IF_ERROR(exprs[i]->execute(&block, &result_col_id)); + } + + // TODO: opt the column is const + block.get_by_position(result_col_id).column = + block.get_by_position(result_col_id).column->convert_to_full_column_if_const(); + res_col_ids[i] = result_col_id; + } + return Status::OK(); +} + +Status HashJoinProbeOperatorX::push(RuntimeState* state, vectorized::Block* input_block, + SourceState source_state) { + auto& local_state = state->get_local_state(id())->cast(); + local_state._probe_eos = source_state == SourceState::FINISHED; + if (input_block->rows() > 0) { + COUNTER_UPDATE(local_state._probe_rows_counter, input_block->rows()); + int probe_expr_ctxs_sz = local_state._probe_expr_ctxs.size(); + local_state._probe_columns.resize(probe_expr_ctxs_sz); + + std::vector res_col_ids(probe_expr_ctxs_sz); + RETURN_IF_ERROR(_do_evaluate(*input_block, local_state._probe_expr_ctxs, + *local_state._probe_expr_call_timer, res_col_ids)); + if (_join_op == TJoinOp::RIGHT_OUTER_JOIN || _join_op == TJoinOp::FULL_OUTER_JOIN) { + local_state._probe_column_convert_to_null = + local_state._dependency->convert_block_to_null(*input_block); + } + // TODO: Now we are not sure whether a column is nullable only by ExecNode's `row_desc` + // so we have to initialize this flag by the first probe block. + if (!local_state._has_set_need_null_map_for_probe) { + local_state._has_set_need_null_map_for_probe = true; + local_state._need_null_map_for_probe = + local_state._need_probe_null_map(*input_block, res_col_ids); + } + if (local_state._need_null_map_for_probe) { + if (local_state._null_map_column == nullptr) { + local_state._null_map_column = vectorized::ColumnUInt8::create(); + } + local_state._null_map_column->get_data().assign(input_block->rows(), (uint8_t)0); + } + + RETURN_IF_ERROR(local_state._dependency->extract_join_column( + *input_block, local_state._null_map_column, local_state._probe_columns, + res_col_ids)); + if (&local_state._probe_block != input_block) { + input_block->swap(local_state._probe_block); + } + } + return Status::OK(); +} + +Status HashJoinProbeOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(JoinProbeOperatorX::init(tnode, state)); + DCHECK(tnode.__isset.hash_join_node); + const bool probe_dispose_null = + _match_all_probe || _build_unique || _join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || + _join_op == TJoinOp::LEFT_ANTI_JOIN || _join_op == TJoinOp::LEFT_SEMI_JOIN; + const std::vector& eq_join_conjuncts = tnode.hash_join_node.eq_join_conjuncts; + std::vector probe_not_ignore_null(eq_join_conjuncts.size()); + size_t conjuncts_index = 0; + for (const auto& eq_join_conjunct : eq_join_conjuncts) { + vectorized::VExprContextSPtr ctx; + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(eq_join_conjunct.left, ctx)); + _probe_expr_ctxs.push_back(ctx); + bool null_aware = eq_join_conjunct.__isset.opcode && + eq_join_conjunct.opcode == TExprOpcode::EQ_FOR_NULL; + probe_not_ignore_null[conjuncts_index] = + null_aware || + (_probe_expr_ctxs.back()->root()->is_nullable() && probe_dispose_null); + conjuncts_index++; + } + for (size_t i = 0; i < _probe_expr_ctxs.size(); ++i) { + _probe_ignore_null |= !probe_not_ignore_null[i]; + } + if (tnode.hash_join_node.__isset.other_join_conjuncts && + !tnode.hash_join_node.other_join_conjuncts.empty()) { + RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees( + tnode.hash_join_node.other_join_conjuncts, _other_join_conjuncts)); + + DCHECK(!_build_unique); + DCHECK(_have_other_join_conjunct); + } else if (tnode.hash_join_node.__isset.vother_join_conjunct) { + _other_join_conjuncts.resize(1); + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree( + tnode.hash_join_node.vother_join_conjunct, _other_join_conjuncts[0])); + + // If LEFT SEMI JOIN/LEFT ANTI JOIN with not equal predicate, + // build table should not be deduplicated. + DCHECK(!_build_unique); + DCHECK(_have_other_join_conjunct); + } + + std::vector hash_output_slot_ids; + // init left/right output slots flags, only column of slot_id in _hash_output_slot_ids need + // insert to output block of hash join. + // _left_output_slots_flags : column of left table need to output set flag = true + // _rgiht_output_slots_flags : column of right table need to output set flag = true + // if _hash_output_slot_ids is empty, means all column of left/right table need to output. + auto init_output_slots_flags = [&hash_output_slot_ids](auto& tuple_descs, + auto& output_slot_flags) { + for (const auto& tuple_desc : tuple_descs) { + for (const auto& slot_desc : tuple_desc->slots()) { + output_slot_flags.emplace_back( + hash_output_slot_ids.empty() || + std::find(hash_output_slot_ids.begin(), hash_output_slot_ids.end(), + slot_desc->id()) != hash_output_slot_ids.end()); + } + } + }; + init_output_slots_flags(_child_x->row_desc().tuple_descriptors(), _left_output_slot_flags); + init_output_slots_flags(_build_side_child->row_desc().tuple_descriptors(), + _right_output_slot_flags); + return Status::OK(); +} + +Status HashJoinProbeOperatorX::prepare(RuntimeState* state) { + RETURN_IF_ERROR(JoinProbeOperatorX::prepare(state)); + RETURN_IF_ERROR(vectorized::VExpr::prepare(_output_expr_ctxs, state, *_intermediate_row_desc)); + // _other_join_conjuncts are evaluated in the context of the rows produced by this node + for (auto& conjunct : _other_join_conjuncts) { + RETURN_IF_ERROR(conjunct->prepare(state, *_intermediate_row_desc)); + } + RETURN_IF_ERROR(vectorized::VExpr::prepare(_probe_expr_ctxs, state, _child_x->row_desc())); + // right table data types + _right_table_data_types = + vectorized::VectorizedUtils::get_data_types(_build_side_child->row_desc()); + _left_table_data_types = vectorized::VectorizedUtils::get_data_types(_child_x->row_desc()); + _build_side_child.reset(); + return Status::OK(); +} + +Status HashJoinProbeOperatorX::open(RuntimeState* state) { + RETURN_IF_ERROR(JoinProbeOperatorX::open(state)); + RETURN_IF_ERROR(vectorized::VExpr::open(_probe_expr_ctxs, state)); + for (auto& conjunct : _other_join_conjuncts) { + RETURN_IF_ERROR(conjunct->open(state)); + } + return Status::OK(); +} + +bool HashJoinProbeOperatorX::can_read(RuntimeState* state) { + auto& local_state = state->get_local_state(id())->cast(); + return local_state._dependency->done(); +} + } // namespace pipeline -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index b59a6435bdc0c45..72c2c1bf2e603be 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -20,6 +20,8 @@ #include "common/status.h" #include "operator.h" +#include "pipeline/exec/join_probe_operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/join/vhash_join_node.h" namespace doris { @@ -44,5 +46,100 @@ class HashJoinProbeOperator final : public StatefulOperator { +public: + using Parent = HashJoinProbeOperatorX; + ENABLE_FACTORY_CREATOR(HashJoinProbeLocalState); + HashJoinProbeLocalState(RuntimeState* state, OperatorXBase* parent); + ~HashJoinProbeLocalState() = default; + + Status init(RuntimeState* state, LocalStateInfo& info) override; + Status close(RuntimeState* state) override; + + void prepare_for_next(); + void add_tuple_is_null_column(vectorized::Block* block); + void init_for_probe(RuntimeState* state); + + HashJoinProbeOperatorX* join_probe() { return (HashJoinProbeOperatorX*)_parent; } + +private: + void _prepare_probe_block(); + bool _need_probe_null_map(vectorized::Block& block, const std::vector& res_col_ids); + friend class HashJoinProbeOperatorX; + friend struct vectorized::HashJoinProbeContext; + + std::unique_ptr _child_block; + SourceState _child_source_state; + + int _probe_index = -1; + bool _probe_eos = false; + std::atomic _probe_inited = false; + + vectorized::Block _probe_block; + vectorized::ColumnRawPtrs _probe_columns; + // other expr + vectorized::VExprContextSPtrs _other_join_conjuncts; + // probe expr + vectorized::VExprContextSPtrs _probe_expr_ctxs; + std::vector _probe_column_disguise_null; + std::vector _probe_column_convert_to_null; + + bool _need_null_map_for_probe = false; + bool _has_set_need_null_map_for_probe = false; + bool _probe_ignore_null = false; + std::unique_ptr _probe_context; + vectorized::ColumnUInt8::MutablePtr _null_map_column; + // for cases when a probe row matches more than batch size build rows. + bool _is_any_probe_match_row_output = false; + std::unique_ptr _process_hashtable_ctx_variants = nullptr; + + RuntimeProfile::Counter* _probe_expr_call_timer; + RuntimeProfile::Counter* _probe_next_timer; + RuntimeProfile::Counter* _probe_side_output_timer; + RuntimeProfile::Counter* _probe_process_hashtable_timer; + RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage; + RuntimeProfile::Counter* _search_hashtable_timer; + RuntimeProfile::Counter* _build_side_output_timer; + RuntimeProfile::Counter* _process_other_join_conjunct_timer; +}; + +class HashJoinProbeOperatorX final : public JoinProbeOperatorX { +public: + HashJoinProbeOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + Status init(const TPlanNode& tnode, RuntimeState* state) override; + Status prepare(RuntimeState* state) override; + Status open(RuntimeState* state) override; + bool can_read(RuntimeState* state) override; + + Status get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) override; + + Status push(RuntimeState* state, vectorized::Block* input_block, SourceState source_state); + Status pull(doris::RuntimeState* state, vectorized::Block* output_block, + SourceState& source_state); + + bool need_more_input_data(RuntimeState* state) const; + +private: + Status _do_evaluate(vectorized::Block& block, vectorized::VExprContextSPtrs& exprs, + RuntimeProfile::Counter& expr_call_timer, std::vector& res_col_ids); + friend class HashJoinProbeLocalState; + friend struct vectorized::HashJoinProbeContext; + + // other expr + vectorized::VExprContextSPtrs _other_join_conjuncts; + // probe expr + vectorized::VExprContextSPtrs _probe_expr_ctxs; + bool _probe_ignore_null = false; + + vectorized::DataTypes _right_table_data_types; + vectorized::DataTypes _left_table_data_types; + std::vector _hash_output_slot_ids; + std::vector _left_output_slot_flags; + std::vector _right_output_slot_flags; +}; + } // namespace pipeline -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/pipeline/exec/join_build_sink_operator.cpp b/be/src/pipeline/exec/join_build_sink_operator.cpp new file mode 100644 index 000000000000000..5de7a2504bb6d4b --- /dev/null +++ b/be/src/pipeline/exec/join_build_sink_operator.cpp @@ -0,0 +1,122 @@ +// 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 "join_build_sink_operator.h" + +#include "pipeline/exec/hashjoin_build_sink.h" +#include "pipeline/pipeline_x/operator.h" + +namespace doris::pipeline { + +template +Status JoinBuildSinkLocalState::init(RuntimeState* state, + LocalSinkStateInfo& info) { + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); + auto& p = PipelineXSinkLocalState::_parent + ->template cast(); + + PipelineXSinkLocalState::profile()->add_info_string("JoinType", + to_string(p._join_op)); + _build_phase_profile = PipelineXSinkLocalState::profile()->create_child( + "BuildPhase", true, true); + _build_get_next_timer = ADD_TIMER(_build_phase_profile, "BuildGetNextTime"); + _build_timer = ADD_TIMER(_build_phase_profile, "BuildTime"); + _build_rows_counter = ADD_COUNTER(_build_phase_profile, "BuildRows", TUnit::UNIT); + + _push_down_timer = ADD_TIMER(PipelineXSinkLocalState::profile(), + "PublishRuntimeFilterTime"); + _push_compute_timer = + ADD_TIMER(PipelineXSinkLocalState::profile(), "PushDownComputeTime"); + + return Status::OK(); +} + +template +JoinBuildSinkOperatorX::JoinBuildSinkOperatorX(ObjectPool* pool, + const TPlanNode& tnode, + const DescriptorTbl& descs) + : DataSinkOperatorX(tnode.node_id), + _join_op(tnode.__isset.hash_join_node ? tnode.hash_join_node.join_op + : (tnode.__isset.nested_loop_join_node + ? tnode.nested_loop_join_node.join_op + : TJoinOp::CROSS_JOIN)), + _have_other_join_conjunct(tnode.__isset.hash_join_node && + ((tnode.hash_join_node.__isset.other_join_conjuncts && + !tnode.hash_join_node.other_join_conjuncts.empty()) || + tnode.hash_join_node.__isset.vother_join_conjunct)), + _match_all_probe(_join_op == TJoinOp::LEFT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN), + _match_all_build(_join_op == TJoinOp::RIGHT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN), + _build_unique(!_have_other_join_conjunct && + (_join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || + _join_op == TJoinOp::LEFT_ANTI_JOIN || + _join_op == TJoinOp::LEFT_SEMI_JOIN)), + _is_right_semi_anti(_join_op == TJoinOp::RIGHT_ANTI_JOIN || + _join_op == TJoinOp::RIGHT_SEMI_JOIN), + _is_left_semi_anti(_join_op == TJoinOp::LEFT_ANTI_JOIN || + _join_op == TJoinOp::LEFT_SEMI_JOIN || + _join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN), + _is_outer_join(_match_all_build || _match_all_probe), + _is_mark_join(tnode.__isset.nested_loop_join_node + ? (tnode.nested_loop_join_node.__isset.is_mark + ? tnode.nested_loop_join_node.is_mark + : false) + : tnode.hash_join_node.__isset.is_mark ? tnode.hash_join_node.is_mark + : false), + _short_circuit_for_null_in_build_side(_join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { + _init_join_op(); + if (_is_mark_join) { + DCHECK(_join_op == TJoinOp::LEFT_ANTI_JOIN || _join_op == TJoinOp::LEFT_SEMI_JOIN || + _join_op == TJoinOp::CROSS_JOIN || _join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) + << "Mark join is only supported for null aware left semi/anti join and cross join " + "but this is " + << _join_op; + } +} + +#define APPLY_FOR_JOINOP_VARIANTS(M) \ + M(INNER_JOIN) \ + M(LEFT_SEMI_JOIN) \ + M(LEFT_ANTI_JOIN) \ + M(LEFT_OUTER_JOIN) \ + M(FULL_OUTER_JOIN) \ + M(RIGHT_OUTER_JOIN) \ + M(CROSS_JOIN) \ + M(RIGHT_SEMI_JOIN) \ + M(RIGHT_ANTI_JOIN) \ + M(NULL_AWARE_LEFT_ANTI_JOIN) + +template +void JoinBuildSinkOperatorX::_init_join_op() { + switch (_join_op) { +#define M(NAME) \ + case TJoinOp::NAME: \ + _join_op_variants.emplace>(); \ + break; + APPLY_FOR_JOINOP_VARIANTS(M); +#undef M + default: + //do nothing + break; + } +} + +template class JoinBuildSinkOperatorX; +template class JoinBuildSinkLocalState; + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/join_build_sink_operator.h b/be/src/pipeline/exec/join_build_sink_operator.h new file mode 100644 index 000000000000000..ff13fae3e9bc2cd --- /dev/null +++ b/be/src/pipeline/exec/join_build_sink_operator.h @@ -0,0 +1,85 @@ +// 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 "operator.h" +#include "pipeline/pipeline_x/dependency.h" +#include "pipeline/pipeline_x/operator.h" +#include "vec/exec/join/vjoin_node_base.h" + +namespace doris { + +namespace pipeline { +template +class JoinBuildSinkOperatorX; + +template +class JoinBuildSinkLocalState : public PipelineXSinkLocalState { +public: + virtual Status init(RuntimeState* state, LocalSinkStateInfo& info) override; + +protected: + JoinBuildSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : PipelineXSinkLocalState(parent, state) {} + virtual ~JoinBuildSinkLocalState() = default; + template + friend class JoinBuildSinkOperatorX; + + bool _short_circuit_for_null_in_probe_side = false; + + RuntimeProfile* _build_phase_profile; + RuntimeProfile::Counter* _build_timer; + RuntimeProfile::Counter* _build_get_next_timer; + RuntimeProfile::Counter* _build_rows_counter; + RuntimeProfile::Counter* _push_down_timer; + RuntimeProfile::Counter* _push_compute_timer; +}; + +template +class JoinBuildSinkOperatorX : public DataSinkOperatorX { +public: + JoinBuildSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + virtual ~JoinBuildSinkOperatorX() = default; + +protected: + void _init_join_op(); + template + friend class JoinBuildSinkLocalState; + + TJoinOp::type _join_op; + vectorized::JoinOpVariants _join_op_variants; + + const bool _have_other_join_conjunct; + const bool _match_all_probe; // output all rows coming from the probe input. Full/Left Join + const bool _match_all_build; // output all rows coming from the build input. Full/Right Join + bool _build_unique; // build a hash table without duplicated rows. Left semi/anti Join + + const bool _is_right_semi_anti; + const bool _is_left_semi_anti; + const bool _is_outer_join; + const bool _is_mark_join; + + // For null aware left anti join, we apply a short circuit strategy. + // 1. Set _short_circuit_for_null_in_build_side to true if join operator is null aware left anti join. + // 2. In build phase, we stop materialize build side when we meet the first null value and set _short_circuit_for_null_in_probe_side to true. + // 3. In probe phase, if _short_circuit_for_null_in_probe_side is true, join node returns empty block directly. Otherwise, probing will continue as the same as generic left anti join. + const bool _short_circuit_for_null_in_build_side; +}; + +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp new file mode 100644 index 000000000000000..3670d640d5dc5b4 --- /dev/null +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -0,0 +1,237 @@ +// 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 "join_probe_operator.h" + +#include "pipeline/exec/hashjoin_probe_operator.h" +#include "pipeline/exec/operator.h" + +namespace doris::pipeline { + +template +Status JoinProbeLocalState::init(RuntimeState* state, + LocalStateInfo& info) { + RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); + auto& p = + PipelineXLocalState::_parent->template cast(); + // only use in outer join as the bool column to mark for function of `tuple_is_null` + if (p._is_outer_join) { + _tuple_is_null_left_flag_column = vectorized::ColumnUInt8::create(); + _tuple_is_null_right_flag_column = vectorized::ColumnUInt8::create(); + } + _output_expr_ctxs.resize(p._output_expr_ctxs.size()); + for (size_t i = 0; i < _output_expr_ctxs.size(); i++) { + RETURN_IF_ERROR(p._output_expr_ctxs[i]->clone(state, _output_expr_ctxs[i])); + } + + _probe_phase_profile = + PipelineXLocalState::profile()->create_child("ProbePhase", true, true); + _probe_timer = ADD_TIMER(_probe_phase_profile, "ProbeTime"); + _join_filter_timer = ADD_CHILD_TIMER(_probe_phase_profile, "JoinFilterTimer", "ProbeTime"); + _build_output_block_timer = + ADD_CHILD_TIMER(_probe_phase_profile, "BuildOutputBlock", "ProbeTime"); + _probe_rows_counter = ADD_COUNTER(_probe_phase_profile, "ProbeRows", TUnit::UNIT); + + return Status::OK(); +} + +template +Status JoinProbeLocalState::close(RuntimeState* state) { + if (PipelineXLocalState::_closed) { + return Status::OK(); + } + _join_block.clear(); + return PipelineXLocalState::close(state); +} + +template +void JoinProbeLocalState::_construct_mutable_join_block() { + auto& p = + PipelineXLocalState::_parent->template cast(); + const auto& mutable_block_desc = p._intermediate_row_desc; + for (const auto tuple_desc : mutable_block_desc->tuple_descriptors()) { + for (const auto slot_desc : tuple_desc->slots()) { + auto type_ptr = slot_desc->get_data_type_ptr(); + _join_block.insert({type_ptr->create_column(), type_ptr, slot_desc->col_name()}); + } + } + if (p._is_mark_join) { + _join_block.replace_by_position( + _join_block.columns() - 1, + remove_nullable(_join_block.get_by_position(_join_block.columns() - 1).column)); + } +} + +template +Status JoinProbeLocalState::_build_output_block( + vectorized::Block* origin_block, vectorized::Block* output_block, bool keep_origin) { + auto& p = + PipelineXLocalState::_parent->template cast(); + SCOPED_TIMER(_build_output_block_timer); + auto is_mem_reuse = output_block->mem_reuse(); + vectorized::MutableBlock mutable_block = + is_mem_reuse ? vectorized::MutableBlock(output_block) + : vectorized::MutableBlock( + vectorized::VectorizedUtils::create_empty_columnswithtypename( + p.row_desc())); + auto rows = origin_block->rows(); + // TODO: After FE plan support same nullable of output expr and origin block and mutable column + // we should replace `insert_column_datas` by `insert_range_from` + + auto insert_column_datas = [keep_origin](auto& to, vectorized::ColumnPtr& from, size_t rows) { + if (to->is_nullable() && !from->is_nullable()) { + if (keep_origin || !from->is_exclusive()) { + auto& null_column = reinterpret_cast(*to); + null_column.get_nested_column().insert_range_from(*from, 0, rows); + null_column.get_null_map_column().get_data().resize_fill(rows, 0); + } else { + to = make_nullable(from, false)->assume_mutable(); + } + } else { + if (keep_origin || !from->is_exclusive()) { + to->insert_range_from(*from, 0, rows); + } else { + to = from->assume_mutable(); + } + } + }; + if (rows != 0) { + auto& mutable_columns = mutable_block.mutable_columns(); + if (_output_expr_ctxs.empty()) { + DCHECK(mutable_columns.size() == p.row_desc().num_materialized_slots()); + for (int i = 0; i < mutable_columns.size(); ++i) { + insert_column_datas(mutable_columns[i], origin_block->get_by_position(i).column, + rows); + } + } else { + DCHECK(mutable_columns.size() == p.row_desc().num_materialized_slots()); + SCOPED_TIMER(PipelineXLocalState::_projection_timer); + for (int i = 0; i < mutable_columns.size(); ++i) { + auto result_column_id = -1; + RETURN_IF_ERROR(_output_expr_ctxs[i]->execute(origin_block, &result_column_id)); + auto& origin_column = origin_block->get_by_position(result_column_id).column; + + /// `convert_to_full_column_if_const` will create a pointer to the origin column if + /// the origin column is not ColumnConst/ColumnArray, this make the column be not + /// exclusive. + /// TODO: maybe need a method to check if a column need to be converted to full + /// column. + if (is_column_const(*origin_column) || + check_column(origin_column)) { + auto column_ptr = origin_column->convert_to_full_column_if_const(); + insert_column_datas(mutable_columns[i], column_ptr, rows); + } else { + insert_column_datas(mutable_columns[i], origin_column, rows); + } + } + } + + if (!is_mem_reuse || !keep_origin) { + output_block->swap(mutable_block.to_block()); + } + DCHECK(output_block->rows() == rows); + } + + return Status::OK(); +} + +template +void JoinProbeLocalState::_reset_tuple_is_null_column() { + if (PipelineXLocalState::_parent->template cast() + ._is_outer_join) { + reinterpret_cast(*_tuple_is_null_left_flag_column).clear(); + reinterpret_cast(*_tuple_is_null_right_flag_column).clear(); + } +} + +template +JoinProbeOperatorX::JoinProbeOperatorX(ObjectPool* pool, const TPlanNode& tnode, + const DescriptorTbl& descs) + : OperatorX(pool, tnode, descs), + _join_op(tnode.__isset.hash_join_node ? tnode.hash_join_node.join_op + : (tnode.__isset.nested_loop_join_node + ? tnode.nested_loop_join_node.join_op + : TJoinOp::CROSS_JOIN)), + _have_other_join_conjunct(tnode.__isset.hash_join_node && + ((tnode.hash_join_node.__isset.other_join_conjuncts && + !tnode.hash_join_node.other_join_conjuncts.empty()) || + tnode.hash_join_node.__isset.vother_join_conjunct)), + _match_all_probe(_join_op == TJoinOp::LEFT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN), + _match_all_build(_join_op == TJoinOp::RIGHT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN), + _build_unique(!_have_other_join_conjunct && + (_join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || + _join_op == TJoinOp::LEFT_ANTI_JOIN || + _join_op == TJoinOp::LEFT_SEMI_JOIN)), + _is_right_semi_anti(_join_op == TJoinOp::RIGHT_ANTI_JOIN || + _join_op == TJoinOp::RIGHT_SEMI_JOIN), + _is_left_semi_anti(_join_op == TJoinOp::LEFT_ANTI_JOIN || + _join_op == TJoinOp::LEFT_SEMI_JOIN || + _join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN), + _is_outer_join(_match_all_build || _match_all_probe), + _is_mark_join(tnode.__isset.nested_loop_join_node + ? (tnode.nested_loop_join_node.__isset.is_mark + ? tnode.nested_loop_join_node.is_mark + : false) + : tnode.hash_join_node.__isset.is_mark ? tnode.hash_join_node.is_mark + : false) { + if (tnode.__isset.hash_join_node) { + _intermediate_row_desc.reset(new RowDescriptor( + descs, tnode.hash_join_node.vintermediate_tuple_id_list, + std::vector(tnode.hash_join_node.vintermediate_tuple_id_list.size()))); + _output_row_desc.reset( + new RowDescriptor(descs, {tnode.hash_join_node.voutput_tuple_id}, {false})); + } else if (tnode.__isset.nested_loop_join_node) { + _intermediate_row_desc.reset(new RowDescriptor( + descs, tnode.nested_loop_join_node.vintermediate_tuple_id_list, + std::vector(tnode.nested_loop_join_node.vintermediate_tuple_id_list.size()))); + _output_row_desc.reset( + new RowDescriptor(descs, {tnode.nested_loop_join_node.voutput_tuple_id}, {false})); + } else { + // Iff BE has been upgraded and FE has not yet, we should keep origin logics for CROSS JOIN. + DCHECK_EQ(_join_op, TJoinOp::CROSS_JOIN); + } +} + +template +Status JoinProbeOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(OperatorX::init(tnode, state)); + if (tnode.__isset.hash_join_node || tnode.__isset.nested_loop_join_node) { + const auto& output_exprs = tnode.__isset.hash_join_node + ? tnode.hash_join_node.srcExprList + : tnode.nested_loop_join_node.srcExprList; + for (const auto& expr : output_exprs) { + vectorized::VExprContextSPtr ctx; + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(expr, ctx)); + _output_expr_ctxs.push_back(ctx); + } + } + + return Status::OK(); +} + +template +Status JoinProbeOperatorX::open(doris::RuntimeState* state) { + RETURN_IF_ERROR(OperatorX::open(state)); + return vectorized::VExpr::open(_output_expr_ctxs, state); +} + +template class JoinProbeLocalState; +template class JoinProbeOperatorX; + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/join_probe_operator.h b/be/src/pipeline/exec/join_probe_operator.h new file mode 100644 index 000000000000000..e39e9a20f549d12 --- /dev/null +++ b/be/src/pipeline/exec/join_probe_operator.h @@ -0,0 +1,108 @@ +// 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 "operator.h" +#include "pipeline/pipeline_x/dependency.h" +#include "pipeline/pipeline_x/operator.h" +#include "vec/exec/join/vjoin_node_base.h" + +namespace doris { + +namespace pipeline { +template +class JoinProbeOperatorX; +template +class JoinProbeLocalState : public PipelineXLocalState { +public: + virtual Status init(RuntimeState* state, LocalStateInfo& info) override; + virtual Status close(RuntimeState* state) override; + +protected: + JoinProbeLocalState(RuntimeState* state, OperatorXBase* parent) + : PipelineXLocalState(state, parent) {} + virtual ~JoinProbeLocalState() = default; + void _construct_mutable_join_block(); + Status _build_output_block(vectorized::Block* origin_block, vectorized::Block* output_block, + bool keep_origin); + void _reset_tuple_is_null_column(); + // output expr + vectorized::VExprContextSPtrs _output_expr_ctxs; + vectorized::Block _join_block; + vectorized::MutableColumnPtr _tuple_is_null_left_flag_column; + vectorized::MutableColumnPtr _tuple_is_null_right_flag_column; + + RuntimeProfile* _probe_phase_profile; + RuntimeProfile::Counter* _probe_timer; + RuntimeProfile::Counter* _probe_rows_counter; + RuntimeProfile::Counter* _join_filter_timer; + RuntimeProfile::Counter* _build_output_block_timer; +}; + +template +class JoinProbeOperatorX : public OperatorX { +public: + JoinProbeOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + virtual Status init(const TPlanNode& tnode, RuntimeState* state) override; + + Status open(doris::RuntimeState* state) override; + [[nodiscard]] const RowDescriptor& row_desc() override { return *_output_row_desc; } + + [[nodiscard]] const RowDescriptor& intermediate_row_desc() const override { + return *_intermediate_row_desc; + } + + [[nodiscard]] bool is_source() const override { return false; } + + void set_build_side_child(OperatorXPtr& build_side_child) { + _build_side_child = build_side_child; + } + + Status set_child(OperatorXPtr child) override { + if (OperatorX::_child_x) { + set_build_side_child(child); + } else { + OperatorX::_child_x = std::move(child); + } + return Status::OK(); + } + +protected: + template + friend class JoinProbeLocalState; + + TJoinOp::type _join_op; + const bool _have_other_join_conjunct; + const bool _match_all_probe; // output all rows coming from the probe input. Full/Left Join + const bool _match_all_build; // output all rows coming from the build input. Full/Right Join + const bool _build_unique; // build a hash table without duplicated rows. Left semi/anti Join + + const bool _is_right_semi_anti; + const bool _is_left_semi_anti; + const bool _is_outer_join; + const bool _is_mark_join; + + std::unique_ptr _output_row_desc; + std::unique_ptr _intermediate_row_desc; + // output expr + vectorized::VExprContextSPtrs _output_expr_ctxs; + OperatorXPtr _build_side_child; +}; + +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp index 5eca2f2b16dc29a..37df7e5ed6b5641 100644 --- a/be/src/pipeline/exec/olap_scan_operator.cpp +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -42,7 +42,7 @@ namespace doris::pipeline { Status OlapScanLocalState::_init_profile() { - RETURN_IF_ERROR(ScanLocalState::_init_profile()); + RETURN_IF_ERROR(ScanLocalState::_init_profile()); // 1. init segment profile _segment_profile.reset(new RuntimeProfile("SegmentIterator")); _scanner_profile->add_child(_segment_profile.get(), true, nullptr); @@ -471,8 +471,9 @@ void OlapScanLocalState::add_filter_info(int id, const PredicateFilterInfo& upda } OlapScanOperatorX::OlapScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs, std::string op_name) - : ScanOperatorX(pool, tnode, descs, op_name), _olap_scan_node(tnode.olap_scan_node) { + const DescriptorTbl& descs) + : ScanOperatorX(pool, tnode, descs), + _olap_scan_node(tnode.olap_scan_node) { _output_tuple_id = tnode.olap_scan_node.tuple_id; _col_distribute_ids = tnode.olap_scan_node.distribute_column_ids; if (_olap_scan_node.__isset.sort_info && _olap_scan_node.__isset.sort_limit) { @@ -480,10 +481,4 @@ OlapScanOperatorX::OlapScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, } } -Status OlapScanOperatorX::setup_local_state(RuntimeState* state, LocalStateInfo& info) { - auto local_state = OlapScanLocalState::create_shared(state, this); - state->emplace_local_state(id(), local_state); - return local_state->init(state, info); -} - } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/olap_scan_operator.h b/be/src/pipeline/exec/olap_scan_operator.h index aec50d5d2f3a87d..dce1f680a921bfa 100644 --- a/be/src/pipeline/exec/olap_scan_operator.h +++ b/be/src/pipeline/exec/olap_scan_operator.h @@ -24,6 +24,7 @@ #include "common/status.h" #include "operator.h" #include "pipeline/exec/scan_operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/scan/vscan_node.h" namespace doris { @@ -37,7 +38,9 @@ class NewOlapScanner; namespace doris::pipeline { class OlapScanOperatorX; -class OlapScanLocalState final : public ScanLocalState { +class OlapScanLocalState final : public ScanLocalState { +public: + using Parent = OlapScanOperatorX; ENABLE_FACTORY_CREATOR(OlapScanLocalState); OlapScanLocalState(RuntimeState* state, OperatorXBase* parent) : ScanLocalState(state, parent) {} @@ -177,12 +180,9 @@ class OlapScanLocalState final : public ScanLocalState { std::mutex _profile_mtx; }; -class OlapScanOperatorX final : public ScanOperatorX { +class OlapScanOperatorX final : public ScanOperatorX { public: - OlapScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string op_name); - - Status setup_local_state(RuntimeState* state, LocalStateInfo& info) override; + OlapScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); private: friend class OlapScanLocalState; diff --git a/be/src/pipeline/exec/operator.cpp b/be/src/pipeline/exec/operator.cpp index 658df5f3cde3677..11643bd4c3693d9 100644 --- a/be/src/pipeline/exec/operator.cpp +++ b/be/src/pipeline/exec/operator.cpp @@ -62,180 +62,4 @@ std::string OperatorBase::debug_string() const { return ss.str(); } -Status PipelineXSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - _mem_tracker = std::make_unique(_parent->get_name()); - return Status::OK(); -} - -std::string OperatorXBase::debug_string() const { - std::stringstream ss; - ss << _op_name << ": is_source: " << is_source(); - ss << ", is_closed: " << _is_closed; - return ss.str(); -} - -Status OperatorXBase::init(const TPlanNode& tnode, RuntimeState* /*state*/) { - _init_runtime_profile(); - - if (tnode.__isset.vconjunct) { - vectorized::VExprContextSPtr context; - RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(tnode.vconjunct, context)); - _conjuncts.emplace_back(context); - } else if (tnode.__isset.conjuncts) { - for (auto& conjunct : tnode.conjuncts) { - vectorized::VExprContextSPtr context; - RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(conjunct, context)); - _conjuncts.emplace_back(context); - } - } - - // create the projections expr - if (tnode.__isset.projections) { - DCHECK(tnode.__isset.output_tuple_id); - RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(tnode.projections, _projections)); - } - return Status::OK(); -} - -Status OperatorXBase::prepare(RuntimeState* state) { - for (auto& conjunct : _conjuncts) { - RETURN_IF_ERROR(conjunct->prepare(state, intermediate_row_desc())); - } - - RETURN_IF_ERROR(vectorized::VExpr::prepare(_projections, state, intermediate_row_desc())); - - if (_child_x && !is_source()) { - RETURN_IF_ERROR(_child_x->prepare(state)); - } - - return Status::OK(); -} - -Status OperatorXBase::open(RuntimeState* state) { - for (auto& conjunct : _conjuncts) { - RETURN_IF_ERROR(conjunct->open(state)); - } - RETURN_IF_ERROR(vectorized::VExpr::open(_projections, state)); - return Status::OK(); -} - -void OperatorXBase::_init_runtime_profile() { - std::stringstream ss; - ss << get_name() << " (id=" << _id << ")"; - _runtime_profile.reset(new RuntimeProfile(ss.str())); - _runtime_profile->set_metadata(_id); -} - -Status OperatorXBase::close(RuntimeState* state) { - if (_child_x && !is_source()) { - RETURN_IF_ERROR(_child_x->close(state)); - } - return state->get_local_state(id())->close(state); -} - -Status PipelineXLocalState::init(RuntimeState* state, LocalStateInfo& /*info*/) { - _runtime_profile.reset(new RuntimeProfile("LocalState " + _parent->get_name())); - _parent->get_runtime_profile()->add_child(_runtime_profile.get(), true, nullptr); - _conjuncts.resize(_parent->_conjuncts.size()); - _projections.resize(_parent->_projections.size()); - for (size_t i = 0; i < _conjuncts.size(); i++) { - RETURN_IF_ERROR(_parent->_conjuncts[i]->clone(state, _conjuncts[i])); - } - for (size_t i = 0; i < _projections.size(); i++) { - RETURN_IF_ERROR(_parent->_projections[i]->clone(state, _projections[i])); - } - DCHECK(_runtime_profile.get() != nullptr); - _rows_returned_counter = ADD_COUNTER(_runtime_profile, "RowsReturned", TUnit::UNIT); - _projection_timer = ADD_TIMER(_runtime_profile, "ProjectionTime"); - _rows_returned_rate = profile()->add_derived_counter( - doris::ExecNode::ROW_THROUGHPUT_COUNTER, TUnit::UNIT_PER_SECOND, - std::bind(&RuntimeProfile::units_per_second, _rows_returned_counter, - profile()->total_time_counter()), - ""); - _mem_tracker = std::make_unique("PipelineXLocalState:" + _runtime_profile->name()); - _memory_used_counter = ADD_LABEL_COUNTER(_runtime_profile, "MemoryUsage"); - _peak_memory_usage_counter = _runtime_profile->AddHighWaterMarkCounter( - "PeakMemoryUsage", TUnit::BYTES, "MemoryUsage"); - return Status::OK(); -} - -void PipelineXLocalState::clear_origin_block() { - _origin_block.clear_column_data(_parent->_row_descriptor.num_materialized_slots()); -} - -Status OperatorXBase::do_projections(RuntimeState* state, vectorized::Block* origin_block, - vectorized::Block* output_block) { - auto local_state = state->get_local_state(id()); - SCOPED_TIMER(local_state->_projection_timer); - using namespace vectorized; - vectorized::MutableBlock mutable_block = - vectorized::VectorizedUtils::build_mutable_mem_reuse_block(output_block, - *_output_row_descriptor); - auto rows = origin_block->rows(); - - if (rows != 0) { - auto& mutable_columns = mutable_block.mutable_columns(); - DCHECK(mutable_columns.size() == _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)); - 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 - if (mutable_columns[i]->is_nullable() xor column_ptr->is_nullable()) { - DCHECK(mutable_columns[i]->is_nullable() && !column_ptr->is_nullable()); - reinterpret_cast(mutable_columns[i].get()) - ->insert_range_from_not_nullable(*column_ptr, 0, rows); - } else { - mutable_columns[i]->insert_range_from(*column_ptr, 0, rows); - } - } - DCHECK(mutable_block.rows() == rows); - } - - return Status::OK(); -} - -Status OperatorXBase::get_next_after_projects(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { - auto local_state = state->get_local_state(id()); - if (_output_row_descriptor) { - local_state->clear_origin_block(); - auto status = get_block(state, &local_state->_origin_block, source_state); - if (UNLIKELY(!status.ok())) return status; - return do_projections(state, &local_state->_origin_block, block); - } - local_state->_peak_memory_usage_counter->set(local_state->_mem_tracker->peak_consumption()); - return get_block(state, block, source_state); -} - -bool PipelineXLocalState::reached_limit() const { - return _parent->_limit != -1 && _num_rows_returned >= _parent->_limit; -} - -void PipelineXLocalState::reached_limit(vectorized::Block* block, bool* eos) { - if (_parent->_limit != -1 and _num_rows_returned + block->rows() >= _parent->_limit) { - block->set_num_rows(_parent->_limit - _num_rows_returned); - *eos = true; - } - - _num_rows_returned += block->rows(); - COUNTER_SET(_rows_returned_counter, _num_rows_returned); -} - -void PipelineXLocalState::reached_limit(vectorized::Block* block, SourceState& source_state) { - if (_parent->_limit != -1 and _num_rows_returned + block->rows() >= _parent->_limit) { - block->set_num_rows(_parent->_limit - _num_rows_returned); - source_state = SourceState::FINISHED; - } - - _num_rows_returned += block->rows(); - COUNTER_SET(_rows_returned_counter, _num_rows_returned); -} - -std::string DataSinkOperatorX::debug_string() const { - std::stringstream ss; - ss << _name << ", is_closed: " << _is_closed; - return ss.str(); -} } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index 2fb484afc00d32b..4ba2aec977fef17 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -90,7 +90,7 @@ enum class SinkState : uint8_t { class OperatorBuilderBase; class OperatorBase; class OperatorXBase; -class DataSinkOperatorX; +class DataSinkOperatorXBase; using OperatorPtr = std::shared_ptr; using Operators = std::vector; @@ -98,7 +98,7 @@ using Operators = std::vector; using OperatorXPtr = std::shared_ptr; using OperatorXs = std::vector; -using DataSinkOperatorXPtr = std::shared_ptr; +using DataSinkOperatorXPtr = std::shared_ptr; using OperatorBuilderPtr = std::shared_ptr; using OperatorBuilders = std::vector; @@ -204,7 +204,7 @@ class OperatorBase { return Status::OK(); } - Status set_child(OperatorXPtr child) { + virtual Status set_child(OperatorXPtr child) { _child_x = std::move(child); return Status::OK(); } @@ -482,363 +482,4 @@ class StatefulOperator : public StreamingOperator { SourceState _child_source_state; }; -// This struct is used only for initializing local state. -struct LocalStateInfo { - const std::vector scan_ranges; - Dependency* dependency; - std::shared_ptr recvr; -}; - -// This struct is used only for initializing local sink state. -struct LocalSinkStateInfo { - const int sender_id; - Dependency* dependency; - std::shared_ptr sender; -}; - -class PipelineXLocalState { -public: - PipelineXLocalState(RuntimeState* state, OperatorXBase* parent) - : _num_rows_returned(0), - _rows_returned_counter(nullptr), - _rows_returned_rate(nullptr), - _memory_used_counter(nullptr), - _peak_memory_usage_counter(nullptr), - _parent(parent), - _state(state) {} - virtual ~PipelineXLocalState() {} - - virtual Status init(RuntimeState* state, LocalStateInfo& info); - virtual Status close(RuntimeState* state) { - if (_closed) { - return Status::OK(); - } - if (_rows_returned_counter != nullptr) { - COUNTER_SET(_rows_returned_counter, _num_rows_returned); - } - profile()->add_to_span(_span); - _closed = true; - return Status::OK(); - } - template - TARGET& cast() { - return reinterpret_cast(*this); - } - template - const TARGET& cast() const { - return reinterpret_cast(*this); - } - - // If use projection, we should clear `_origin_block`. - void clear_origin_block(); - - bool reached_limit() const; - void reached_limit(vectorized::Block* block, bool* eos); - void reached_limit(vectorized::Block* block, SourceState& source_state); - RuntimeProfile* profile() { return _runtime_profile.get(); } - - MemTracker* mem_tracker() { return _mem_tracker.get(); } - RuntimeProfile::Counter* rows_returned_counter() { return _rows_returned_counter; } - RuntimeProfile::Counter* rows_returned_rate() { return _rows_returned_rate; } - RuntimeProfile::Counter* memory_used_counter() { return _memory_used_counter; } - RuntimeProfile::Counter* projection_timer() { return _projection_timer; } - - OperatorXBase* parent() { return _parent; } - RuntimeState* state() { return _state; } - vectorized::VExprContextSPtrs& conjuncts() { return _conjuncts; } - vectorized::VExprContextSPtrs& projections() { return _projections; } - int64_t num_rows_returned() const { return _num_rows_returned; } - void add_num_rows_returned(int64_t delta) { _num_rows_returned += delta; } - void set_num_rows_returned(int64_t value) { _num_rows_returned = value; } - -protected: - friend class OperatorXBase; - - ObjectPool* _pool; - int64_t _num_rows_returned; - - std::unique_ptr _runtime_profile; - - // Record this node memory size. it is expected that artificial guarantees are accurate, - // which will providea reference for operator memory. - std::unique_ptr _mem_tracker; - - RuntimeProfile::Counter* _rows_returned_counter; - RuntimeProfile::Counter* _rows_returned_rate; - // Account for peak memory used by this node - RuntimeProfile::Counter* _memory_used_counter; - RuntimeProfile::Counter* _projection_timer; - // Account for peak memory used by this node - RuntimeProfile::Counter* _peak_memory_usage_counter; - - OpentelemetrySpan _span; - OperatorXBase* _parent; - RuntimeState* _state; - vectorized::VExprContextSPtrs _conjuncts; - vectorized::VExprContextSPtrs _projections; - bool _init = false; - bool _closed = false; - vectorized::Block _origin_block; -}; - -class OperatorXBase : public OperatorBase { -public: - OperatorXBase(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string op_name) - : OperatorBase(nullptr), - _id(tnode.node_id), - _type(tnode.node_type), - _pool(pool), - _tuple_ids(tnode.row_tuples), - _row_descriptor(descs, tnode.row_tuples, tnode.nullable_tuples), - _resource_profile(tnode.resource_profile), - _limit(tnode.limit), - _op_name(op_name) { - if (tnode.__isset.output_tuple_id) { - _output_row_descriptor.reset(new RowDescriptor(descs, {tnode.output_tuple_id}, {true})); - } - } - - virtual Status init(const TPlanNode& tnode, RuntimeState* state); - Status init(const TDataSink& tsink) override { - LOG(FATAL) << "should not reach here!"; - return Status::OK(); - } - [[nodiscard]] RuntimeProfile* get_runtime_profile() const override { - return _runtime_profile.get(); - } - [[nodiscard]] std::string get_name() const override { return _op_name; } - - virtual Status prepare(RuntimeState* state) override; - - virtual Status open(RuntimeState* state) override; - - Status finalize(RuntimeState* state) override { return Status::OK(); } - - [[nodiscard]] bool can_terminate_early() override { return false; } - - [[nodiscard]] virtual bool can_terminate_early(RuntimeState* state) { return false; } - - bool can_read() override { - LOG(FATAL) << "should not reach here!"; - return false; - } - - bool can_write() override { - LOG(FATAL) << "should not reach here!"; - return false; - } - - bool is_pending_finish() const override { - LOG(FATAL) << "should not reach here!"; - return false; - } - - Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override { - LOG(FATAL) << "should not reach here!"; - return Status::OK(); - } - - virtual Status close(RuntimeState* state) override; - - virtual bool can_read(RuntimeState* state) { return false; } - - virtual bool is_pending_finish(RuntimeState* state) const { return false; } - - [[nodiscard]] virtual const RowDescriptor& intermediate_row_desc() const { - return _row_descriptor; - } - - virtual std::string debug_string() const override; - - virtual Status setup_local_state(RuntimeState* state, LocalStateInfo& info) = 0; - - template - TARGET& cast() { - DCHECK(dynamic_cast(this)); - return reinterpret_cast(*this); - } - template - const TARGET& cast() const { - DCHECK(dynamic_cast(this)); - return reinterpret_cast(*this); - } - - OperatorXPtr get_child() { return _child_x; } - - vectorized::VExprContextSPtrs& conjuncts() { return _conjuncts; } - RowDescriptor& row_descriptor() { return _row_descriptor; } - - [[nodiscard]] int id() const override { return _id; } - - [[nodiscard]] int64_t limit() const { return _limit; } - - const RowDescriptor& row_desc() override { - return _output_row_descriptor ? *_output_row_descriptor : _row_descriptor; - } - - virtual bool is_source() const override { return false; } - [[nodiscard]] virtual bool need_to_create_exch_recv() const { return false; } - - Status get_next_after_projects(RuntimeState* state, vectorized::Block* block, - SourceState& source_state); - - /// Only use in vectorized exec engine try to do projections to trans _row_desc -> _output_row_desc - Status do_projections(RuntimeState* state, vectorized::Block* origin_block, - vectorized::Block* output_block); - -protected: - friend class PipelineXLocalState; - int _id; // unique w/in single plan tree - TPlanNodeType::type _type; - ObjectPool* _pool; - std::vector _tuple_ids; - - vectorized::VExprContextSPtrs _conjuncts; - - RowDescriptor _row_descriptor; - - std::unique_ptr _output_row_descriptor; - vectorized::VExprContextSPtrs _projections; - - /// Resource information sent from the frontend. - const TBackendResourceProfile _resource_profile; - - int64_t _limit; // -1: no limit - std::unique_ptr _runtime_profile; - -private: - void _init_runtime_profile(); - - std::string _op_name; -}; - -class DataSinkOperatorX; - -class PipelineXSinkLocalState { -public: - PipelineXSinkLocalState(DataSinkOperatorX* parent_, RuntimeState* state_) - : _parent(parent_), _state(state_) {} - virtual ~PipelineXSinkLocalState() {} - - virtual Status init(RuntimeState* state, LocalSinkStateInfo& info); - virtual Status close(RuntimeState* state) { - if (_closed) { - return Status::OK(); - } - _closed = true; - return Status::OK(); - } - template - TARGET& cast() { - DCHECK(dynamic_cast(this)); - return reinterpret_cast(*this); - } - template - const TARGET& cast() const { - DCHECK(dynamic_cast(this)); - return reinterpret_cast(*this); - } - - DataSinkOperatorX* parent() { return _parent; } - RuntimeState* state() { return _state; } - RuntimeProfile* profile() { return _profile; } - MemTracker* mem_tracker() { return _mem_tracker.get(); } - QueryStatistics* query_statistics() { return _query_statistics.get(); } - -protected: - DataSinkOperatorX* _parent; - RuntimeState* _state; - RuntimeProfile* _profile; - std::unique_ptr _mem_tracker; - // Maybe this will be transferred to BufferControlBlock. - std::shared_ptr _query_statistics; - // Set to true after close() has been called. subclasses should check and set this in - // close(). - bool _closed = false; -}; - -class DataSinkOperatorX : public OperatorBase { -public: - DataSinkOperatorX(const int id) : OperatorBase(nullptr), _id(id) {} - - ~DataSinkOperatorX() override = default; - - // For agg/sort/join sink. - virtual Status init(const TPlanNode& tnode, RuntimeState* state) { - return Status::InternalError("{} should not init with TPlanNode", _name); - } - - virtual Status init(const TDataSink& tsink) override { return Status::OK(); } - - virtual Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) = 0; - - [[nodiscard]] virtual bool need_to_create_result_sender() const { return false; } - - template - TARGET& cast() { - DCHECK(dynamic_cast(this)); - return reinterpret_cast(*this); - } - template - const TARGET& cast() const { - DCHECK(dynamic_cast(this)); - return reinterpret_cast(*this); - } - - virtual void get_dependency(DependencySPtr& dependency) { - dependency.reset((Dependency*)nullptr); - } - - virtual Status close(RuntimeState* state) override { - return state->get_sink_local_state(id())->close(state); - } - - bool can_read() override { - LOG(FATAL) << "should not reach here!"; - return false; - } - - bool can_write() override { - LOG(FATAL) << "should not reach here!"; - return false; - } - - bool is_pending_finish() const override { - LOG(FATAL) << "should not reach here!"; - return false; - } - - virtual bool can_write(RuntimeState* state) { return false; } - - virtual bool is_pending_finish(RuntimeState* state) const { return false; } - - virtual std::string debug_string() const override; - - bool is_sink() const override { return true; } - - bool is_source() const override { return false; } - - virtual Status close(RuntimeState* state, Status exec_status) { return Status::OK(); } - - [[nodiscard]] RuntimeProfile* get_runtime_profile() const override { return _profile; } - - [[nodiscard]] int id() const override { return _id; } - - [[nodiscard]] std::string get_name() const override { return _name; } - - Status finalize(RuntimeState* state) override { return Status::OK(); } - -protected: - const int _id; - std::string _name; - - // Maybe this will be transferred to BufferControlBlock. - std::shared_ptr _query_statistics; - - OpentelemetrySpan _span {}; - - RuntimeProfile* _profile = nullptr; -}; - } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp index ce8a933d7e2a217..bde0b1574f0304e 100644 --- a/be/src/pipeline/exec/result_sink_operator.cpp +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -51,12 +51,9 @@ bool ResultSinkOperator::can_write() { } Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(PipelineXSinkLocalState<>::init(state, info)); auto& p = _parent->cast(); auto fragment_instance_id = state->fragment_instance_id(); - auto title = fmt::format("VDataBufferSender (dst_fragment_instance_id={:x}-{:x})", - fragment_instance_id.hi, fragment_instance_id.lo); - // create profile - _profile = state->obj_pool()->add(new RuntimeProfile(title)); // create sender _sender = info.sender; _output_vexpr_ctxs.resize(p._output_vexpr_ctxs.size()); @@ -114,12 +111,6 @@ Status ResultSinkOperatorX::open(RuntimeState* state) { return vectorized::VExpr::open(_output_vexpr_ctxs, state); } -Status ResultSinkOperatorX::setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) { - auto local_state = ResultSinkLocalState::create_shared(this, state); - state->emplace_sink_local_state(id(), local_state); - return local_state->init(state, info); -} - Status ResultSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) { auto& local_state = state->get_sink_local_state(id())->cast(); @@ -175,7 +166,7 @@ Status ResultSinkLocalState::close(RuntimeState* state) { state->exec_env()->result_mgr()->cancel_at_time( time(nullptr) + config::result_buffer_cancelled_interval_time, state->fragment_instance_id()); - RETURN_IF_ERROR(PipelineXSinkLocalState::close(state)); + RETURN_IF_ERROR(PipelineXSinkLocalState<>::close(state)); return final_status; } diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index e98bae86e7cfcc4..19c3b13c0c8ebae 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -20,6 +20,7 @@ #include #include "operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/sink/vresult_sink.h" namespace doris { @@ -41,12 +42,12 @@ class ResultSinkOperator final : public DataSinkOperator { ENABLE_FACTORY_CREATOR(ResultSinkLocalState); public: - ResultSinkLocalState(DataSinkOperatorX* parent, RuntimeState* state) - : PipelineXSinkLocalState(parent, state) {} + ResultSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : PipelineXSinkLocalState<>(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; @@ -59,16 +60,14 @@ class ResultSinkLocalState final : public PipelineXSinkLocalState { std::shared_ptr _sender; std::shared_ptr _writer; - RuntimeProfile* _profile; // Allocated from _pool }; -class ResultSinkOperatorX final : public DataSinkOperatorX { +class ResultSinkOperatorX final : public DataSinkOperatorX { public: ResultSinkOperatorX(const RowDescriptor& row_desc, const std::vector& select_exprs, const TResultSink& sink); Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; - Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) override; Status sink(RuntimeState* state, vectorized::Block* in_block, SourceState source_state) override; diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index 236421ae13f49d3..826c86cd02fa4d0 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -21,6 +21,7 @@ #include +#include "pipeline/exec/olap_scan_operator.h" #include "pipeline/exec/operator.h" #include "vec/exec/runtime_filter_consumer.h" #include "vec/exec/scan/pip_scanner_context.h" @@ -95,30 +96,25 @@ std::string ScanOperator::debug_string() const { return; \ } -ScanLocalState::ScanLocalState(RuntimeState* state_, OperatorXBase* parent_) - : PipelineXLocalState(state_, parent_), - vectorized::RuntimeFilterConsumer(_parent->id(), - _parent->cast().runtime_filter_descs(), - _parent->row_descriptor(), _parent->conjuncts()) {} +template +ScanLocalState::ScanLocalState(RuntimeState* state_, OperatorXBase* parent_) + : ScanLocalStateBase(state_, parent_) {} -bool ScanLocalState::ready_to_read() { +template +bool ScanLocalState::ready_to_read() { return !_scanner_ctx->empty_in_queue(0); } -bool ScanLocalState::should_run_serial() const { - return _parent->cast()._should_run_serial; +template +bool ScanLocalState::should_run_serial() const { + return _parent->cast()._should_run_serial; } -Status ScanLocalState::init(RuntimeState* state, LocalStateInfo& info) { - if (_init) { - return Status::OK(); - } - - auto& p = _parent->cast(); - +template +Status ScanLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); + auto& p = _parent->cast(); set_scan_ranges(info.scan_ranges); - - RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); _common_expr_ctxs_push_down.resize(p._common_expr_ctxs_push_down.size()); for (size_t i = 0; i < _common_expr_ctxs_push_down.size(); i++) { RETURN_IF_ERROR( @@ -154,13 +150,12 @@ Status ScanLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(_scanner_ctx->init()); RETURN_IF_ERROR(state->exec_env()->scanner_scheduler()->submit(_scanner_ctx.get())); } - RETURN_IF_ERROR(status); - _init = true; - return Status::OK(); + return status; } -Status ScanLocalState::_normalize_conjuncts() { - auto& p = _parent->cast(); +template +Status ScanLocalState::_normalize_conjuncts() { + auto& p = _parent->cast(); // The conjuncts is always on output tuple, so use _output_tuple_desc; std::vector slots = p._output_tuple_desc->slots(); @@ -246,9 +241,10 @@ Status ScanLocalState::_normalize_conjuncts() { return Status::OK(); } -Status ScanLocalState::_normalize_predicate(const vectorized::VExprSPtr& conjunct_expr_root, - vectorized::VExprContext* context, - vectorized::VExprSPtr& output_expr) { +template +Status ScanLocalState::_normalize_predicate( + const vectorized::VExprSPtr& conjunct_expr_root, vectorized::VExprContext* context, + vectorized::VExprSPtr& output_expr) { static constexpr auto is_leaf = [](auto&& expr) { return !expr->is_and_expr(); }; auto in_predicate_checker = [](const vectorized::VExprSPtrs& children, std::shared_ptr& slot, @@ -399,10 +395,11 @@ Status ScanLocalState::_normalize_predicate(const vectorized::VExprSPtr& conjunc return Status::OK(); } -Status ScanLocalState::_normalize_bloom_filter(vectorized::VExpr* expr, - vectorized::VExprContext* expr_ctx, - SlotDescriptor* slot, - vectorized::VScanNode::PushDownType* pdt) { +template +Status ScanLocalState::_normalize_bloom_filter(vectorized::VExpr* expr, + vectorized::VExprContext* expr_ctx, + SlotDescriptor* slot, + vectorized::VScanNode::PushDownType* pdt) { if (TExprNodeType::BLOOM_PRED == expr->node_type()) { DCHECK(expr->children().size() == 1); vectorized::VScanNode::PushDownType temp_pdt = _should_push_down_bloom_filter(); @@ -415,10 +412,11 @@ Status ScanLocalState::_normalize_bloom_filter(vectorized::VExpr* expr, return Status::OK(); } -Status ScanLocalState::_normalize_bitmap_filter(vectorized::VExpr* expr, - vectorized::VExprContext* expr_ctx, - SlotDescriptor* slot, - vectorized::VScanNode::PushDownType* pdt) { +template +Status ScanLocalState::_normalize_bitmap_filter(vectorized::VExpr* expr, + vectorized::VExprContext* expr_ctx, + SlotDescriptor* slot, + vectorized::VScanNode::PushDownType* pdt) { if (TExprNodeType::BITMAP_PRED == expr->node_type()) { DCHECK(expr->children().size() == 1); vectorized::VScanNode::PushDownType temp_pdt = _should_push_down_bitmap_filter(); @@ -431,10 +429,10 @@ Status ScanLocalState::_normalize_bitmap_filter(vectorized::VExpr* expr, return Status::OK(); } -Status ScanLocalState::_normalize_function_filters(vectorized::VExpr* expr, - vectorized::VExprContext* expr_ctx, - SlotDescriptor* slot, - vectorized::VScanNode::PushDownType* pdt) { +template +Status ScanLocalState::_normalize_function_filters( + vectorized::VExpr* expr, vectorized::VExprContext* expr_ctx, SlotDescriptor* slot, + vectorized::VScanNode::PushDownType* pdt) { bool opposite = false; vectorized::VExpr* fn_expr = expr; if (TExprNodeType::COMPOUND_PRED == expr->node_type() && @@ -459,7 +457,8 @@ Status ScanLocalState::_normalize_function_filters(vectorized::VExpr* expr, return Status::OK(); } -bool ScanLocalState::_is_predicate_acting_on_slot( +template +bool ScanLocalState::_is_predicate_acting_on_slot( vectorized::VExpr* expr, const std::function&, vectorized::VExprSPtr&)>& @@ -495,7 +494,8 @@ bool ScanLocalState::_is_predicate_acting_on_slot( return true; } -bool ScanLocalState::_ignore_cast(SlotDescriptor* slot, vectorized::VExpr* expr) { +template +bool ScanLocalState::_ignore_cast(SlotDescriptor* slot, vectorized::VExpr* expr) { if (slot->type().is_date_type() && expr->type().is_date_type()) { return true; } @@ -516,9 +516,10 @@ bool ScanLocalState::_ignore_cast(SlotDescriptor* slot, vectorized::VExpr* expr) return false; } -Status ScanLocalState::_eval_const_conjuncts(vectorized::VExpr* vexpr, - vectorized::VExprContext* expr_ctx, - vectorized::VScanNode::PushDownType* pdt) { +template +Status ScanLocalState::_eval_const_conjuncts(vectorized::VExpr* vexpr, + vectorized::VExprContext* expr_ctx, + vectorized::VScanNode::PushDownType* pdt) { char* constant_val = nullptr; if (vexpr->is_constant()) { std::shared_ptr const_col_wrapper; @@ -561,12 +562,11 @@ Status ScanLocalState::_eval_const_conjuncts(vectorized::VExpr* vexpr, return Status::OK(); } +template template -Status ScanLocalState::_normalize_in_and_eq_predicate(vectorized::VExpr* expr, - vectorized::VExprContext* expr_ctx, - SlotDescriptor* slot, - ColumnValueRange& range, - vectorized::VScanNode::PushDownType* pdt) { +Status ScanLocalState::_normalize_in_and_eq_predicate( + vectorized::VExpr* expr, vectorized::VExprContext* expr_ctx, SlotDescriptor* slot, + ColumnValueRange& range, vectorized::VScanNode::PushDownType* pdt) { auto temp_range = ColumnValueRange::create_empty_column_value_range( slot->is_nullable(), slot->type().precision, slot->type().scale); // 1. Normalize in conjuncts like 'where col in (v1, v2, v3)' @@ -577,7 +577,7 @@ Status ScanLocalState::_normalize_in_and_eq_predicate(vectorized::VExpr* expr, if (hybrid_set != nullptr) { // runtime filter produce VDirectInPredicate if (hybrid_set->size() <= - _parent->cast()._max_pushdown_conditions_per_column) { + _parent->cast()._max_pushdown_conditions_per_column) { iter = hybrid_set->begin(); } else { _filter_predicates.in_filters.emplace_back(slot->col_name(), expr->get_set_func()); @@ -662,7 +662,8 @@ Status ScanLocalState::_normalize_in_and_eq_predicate(vectorized::VExpr* expr, return Status::OK(); } -Status ScanLocalState::_should_push_down_binary_predicate( +template +Status ScanLocalState::_should_push_down_binary_predicate( vectorized::VectorizedFnCall* fn_call, vectorized::VExprContext* expr_ctx, StringRef* constant_val, int* slot_ref_child, const std::function& fn_checker, @@ -702,7 +703,8 @@ Status ScanLocalState::_should_push_down_binary_predicate( return Status::OK(); } -vectorized::VScanNode::PushDownType ScanLocalState::_should_push_down_in_predicate( +template +vectorized::VScanNode::PushDownType ScanLocalState::_should_push_down_in_predicate( vectorized::VInPredicate* pred, vectorized::VExprContext* expr_ctx, bool is_not_in) { if (pred->is_not_in() != is_not_in) { return vectorized::VScanNode::PushDownType::UNACCEPTABLE; @@ -710,8 +712,9 @@ vectorized::VScanNode::PushDownType ScanLocalState::_should_push_down_in_predica return vectorized::VScanNode::PushDownType::ACCEPTABLE; } +template template -Status ScanLocalState::_normalize_not_in_and_not_eq_predicate( +Status ScanLocalState::_normalize_not_in_and_not_eq_predicate( vectorized::VExpr* expr, vectorized::VExprContext* expr_ctx, SlotDescriptor* slot, ColumnValueRange& range, vectorized::VScanNode::PushDownType* pdt) { bool is_fixed_range = range.is_fixed_value_range(); @@ -804,7 +807,7 @@ Status ScanLocalState::_normalize_not_in_and_not_eq_predicate( if (is_fixed_range || not_in_range.get_fixed_value_size() <= - _parent->cast()._max_pushdown_conditions_per_column) { + _parent->cast()._max_pushdown_conditions_per_column) { if (!is_fixed_range) { _not_in_value_ranges.push_back(not_in_range); } @@ -813,10 +816,13 @@ Status ScanLocalState::_normalize_not_in_and_not_eq_predicate( return Status::OK(); } +template template -Status ScanLocalState::_change_value_range(ColumnValueRange& temp_range, void* value, - const ChangeFixedValueRangeFunc& func, - const std::string& fn_name, int slot_ref_child) { +Status ScanLocalState::_change_value_range(ColumnValueRange& temp_range, + void* value, + const ChangeFixedValueRangeFunc& func, + const std::string& fn_name, + int slot_ref_child) { if constexpr (PrimitiveType == TYPE_DATE) { vectorized::VecDateTimeValue tmp_value; memcpy(&tmp_value, value, sizeof(vectorized::VecDateTimeValue)); @@ -867,12 +873,11 @@ Status ScanLocalState::_change_value_range(ColumnValueRange& temp return Status::OK(); } +template template -Status ScanLocalState::_normalize_is_null_predicate(vectorized::VExpr* expr, - vectorized::VExprContext* expr_ctx, - SlotDescriptor* slot, - ColumnValueRange& range, - vectorized::VScanNode::PushDownType* pdt) { +Status ScanLocalState::_normalize_is_null_predicate( + vectorized::VExpr* expr, vectorized::VExprContext* expr_ctx, SlotDescriptor* slot, + ColumnValueRange& range, vectorized::VScanNode::PushDownType* pdt) { vectorized::VScanNode::PushDownType temp_pdt = _should_push_down_is_null_predicate(); if (temp_pdt == vectorized::VScanNode::PushDownType::UNACCEPTABLE) { return Status::OK(); @@ -898,12 +903,11 @@ Status ScanLocalState::_normalize_is_null_predicate(vectorized::VExpr* expr, return Status::OK(); } +template template -Status ScanLocalState::_normalize_noneq_binary_predicate(vectorized::VExpr* expr, - vectorized::VExprContext* expr_ctx, - SlotDescriptor* slot, - ColumnValueRange& range, - vectorized::VScanNode::PushDownType* pdt) { +Status ScanLocalState::_normalize_noneq_binary_predicate( + vectorized::VExpr* expr, vectorized::VExprContext* expr_ctx, SlotDescriptor* slot, + ColumnValueRange& range, vectorized::VScanNode::PushDownType* pdt) { if (TExprNodeType::BINARY_PRED == expr->node_type()) { DCHECK(expr->children().size() == 2); @@ -941,7 +945,8 @@ Status ScanLocalState::_normalize_noneq_binary_predicate(vectorized::VExpr* expr return Status::OK(); } -Status ScanLocalState::_normalize_compound_predicate( +template +Status ScanLocalState::_normalize_compound_predicate( vectorized::VExpr* expr, vectorized::VExprContext* expr_ctx, vectorized::VScanNode::PushDownType* pdt, bool _is_runtime_filter_predicate, const std::function template -Status ScanLocalState::_normalize_binary_in_compound_predicate( +Status ScanLocalState::_normalize_binary_in_compound_predicate( vectorized::VExpr* expr, vectorized::VExprContext* expr_ctx, SlotDescriptor* slot, ColumnValueRange& range, vectorized::VScanNode::PushDownType* pdt) { DCHECK(expr->children().size() == 2); @@ -1068,8 +1074,9 @@ Status ScanLocalState::_normalize_binary_in_compound_predicate( return Status::OK(); } +template template -Status ScanLocalState::_normalize_match_in_compound_predicate( +Status ScanLocalState::_normalize_match_in_compound_predicate( vectorized::VExpr* expr, vectorized::VExprContext* expr_ctx, SlotDescriptor* slot, ColumnValueRange& range, vectorized::VScanNode::PushDownType* pdt) { DCHECK(expr->children().size() == 2); @@ -1080,11 +1087,11 @@ Status ScanLocalState::_normalize_match_in_compound_predicate( return Status::OK(); } +template template -Status ScanLocalState::_normalize_match_predicate(vectorized::VExpr* expr, - vectorized::VExprContext* expr_ctx, - SlotDescriptor* slot, ColumnValueRange& range, - vectorized::VScanNode::PushDownType* pdt) { +Status ScanLocalState::_normalize_match_predicate( + vectorized::VExpr* expr, vectorized::VExprContext* expr_ctx, SlotDescriptor* slot, + ColumnValueRange& range, vectorized::VScanNode::PushDownType* pdt) { if (TExprNodeType::MATCH_PRED == expr->node_type()) { DCHECK(expr->children().size() == 2); @@ -1123,7 +1130,8 @@ Status ScanLocalState::_normalize_match_predicate(vectorized::VExpr* expr, return Status::OK(); } -Status ScanLocalState::_prepare_scanners(const int query_parallel_instance_num) { +template +Status ScanLocalState::_prepare_scanners(const int query_parallel_instance_num) { std::list scanners; RETURN_IF_ERROR(_init_scanners(&scanners)); if (scanners.empty()) { @@ -1135,31 +1143,37 @@ Status ScanLocalState::_prepare_scanners(const int query_parallel_instance_num) return Status::OK(); } -Status ScanLocalState::_start_scanners(const std::list& scanners, - const int query_parallel_instance_num) { - auto& p = _parent->cast(); +template +Status ScanLocalState::_start_scanners(const std::list& scanners, + const int query_parallel_instance_num) { + auto& p = _parent->cast(); _scanner_ctx = PipScannerContext::create_shared(state(), this, p._output_tuple_desc, scanners, p.limit(), state()->scan_queue_mem_limit(), p._col_distribute_ids, 1); return Status::OK(); } -const TupleDescriptor* ScanLocalState::input_tuple_desc() const { - return _parent->cast()._input_tuple_desc; +template +const TupleDescriptor* ScanLocalState::input_tuple_desc() const { + return _parent->cast()._input_tuple_desc; } -const TupleDescriptor* ScanLocalState::output_tuple_desc() const { - return _parent->cast()._output_tuple_desc; +template +const TupleDescriptor* ScanLocalState::output_tuple_desc() const { + return _parent->cast()._output_tuple_desc; } -TPushAggOp::type ScanLocalState::get_push_down_agg_type() { - return _parent->cast()._push_down_agg_type; +template +TPushAggOp::type ScanLocalState::get_push_down_agg_type() { + return _parent->cast()._push_down_agg_type; } -int64_t ScanLocalState::limit_per_scanner() { - return _parent->cast()._limit_per_scanner; +template +int64_t ScanLocalState::limit_per_scanner() { + return _parent->cast()._limit_per_scanner; } -Status ScanLocalState::clone_conjunct_ctxs(vectorized::VExprContextSPtrs& conjuncts) { +template +Status ScanLocalState::clone_conjunct_ctxs(vectorized::VExprContextSPtrs& conjuncts) { if (!_conjuncts.empty()) { std::unique_lock l(_rf_locks); conjuncts.resize(_conjuncts.size()); @@ -1170,7 +1184,8 @@ Status ScanLocalState::clone_conjunct_ctxs(vectorized::VExprContextSPtrs& conjun return Status::OK(); } -Status ScanLocalState::_init_profile() { +template +Status ScanLocalState::_init_profile() { // 1. counters for scan node _rows_read_counter = ADD_COUNTER(_runtime_profile, "RowsRead", TUnit::UNIT); _total_throughput_counter = @@ -1208,9 +1223,11 @@ Status ScanLocalState::_init_profile() { return Status::OK(); } -ScanOperatorX::ScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string op_name) - : OperatorXBase(pool, tnode, descs, op_name), _runtime_filter_descs(tnode.runtime_filters) { +template +ScanOperatorX::ScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, + const DescriptorTbl& descs) + : OperatorX(pool, tnode, descs), + _runtime_filter_descs(tnode.runtime_filters) { if (!tnode.__isset.conjuncts || tnode.conjuncts.empty()) { // Which means the request could be fullfilled in a single segment iterator request. if (tnode.limit > 0 && tnode.limit < 1024) { @@ -1219,33 +1236,32 @@ ScanOperatorX::ScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, const Des } } -bool ScanOperatorX::can_read(RuntimeState* state) { - auto& local_state = state->get_local_state(id())->cast(); - if (!local_state._init) { +template +bool ScanOperatorX::can_read(RuntimeState* state) { + auto& local_state = state->get_local_state(id())->template cast(); + if (local_state._eos || local_state._scanner_ctx->done()) { + // _eos: need eos + // _scanner_ctx->done(): need finish + // _scanner_ctx->no_schedule(): should schedule _scanner_ctx return true; } else { - if (local_state._eos || local_state._scanner_ctx->done()) { - // _eos: need eos - // _scanner_ctx->done(): need finish - // _scanner_ctx->no_schedule(): should schedule _scanner_ctx - return true; - } else { - if (local_state._scanner_ctx->get_num_running_scanners() == 0 && - local_state._scanner_ctx->has_enough_space_in_blocks_queue()) { - local_state._scanner_ctx->reschedule_scanner_ctx(); - } - return local_state.ready_to_read(); // there are some blocks to process + if (local_state._scanner_ctx->get_num_running_scanners() == 0 && + local_state._scanner_ctx->has_enough_space_in_blocks_queue()) { + local_state._scanner_ctx->reschedule_scanner_ctx(); } + return local_state.ready_to_read(); // there are some blocks to process } } -bool ScanOperatorX::is_pending_finish(RuntimeState* state) const { - auto& local_state = state->get_local_state(id())->cast(); +template +bool ScanOperatorX::is_pending_finish(RuntimeState* state) const { + auto& local_state = state->get_local_state(id())->template cast(); return local_state._scanner_ctx && !local_state._scanner_ctx->no_schedule(); } -Status ScanOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { - RETURN_IF_ERROR(OperatorXBase::init(tnode, state)); +template +Status ScanOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(OperatorX::init(tnode, state)); const TQueryOptions& query_options = state->query_options(); if (query_options.__isset.max_scan_key_num) { @@ -1273,17 +1289,19 @@ Status ScanOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { return Status::OK(); } -Status ScanOperatorX::open(RuntimeState* state) { +template +Status ScanOperatorX::open(RuntimeState* state) { _input_tuple_desc = state->desc_tbl().get_tuple_descriptor(_input_tuple_id); _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_output_tuple_id); - RETURN_IF_ERROR(OperatorXBase::open(state)); + RETURN_IF_ERROR(OperatorX::open(state)); RETURN_IF_CANCELLED(state); return Status::OK(); } -Status ScanOperatorX::try_close(RuntimeState* state) { - auto& local_state = state->get_local_state(id())->cast(); +template +Status ScanOperatorX::try_close(RuntimeState* state) { + auto& local_state = state->get_local_state(id())->template cast(); if (local_state._scanner_ctx.get()) { // mark this scanner ctx as should_stop to make sure scanners will not be scheduled anymore // TODO: there is a lock in `set_should_stop` may cause some slight impact @@ -1292,19 +1310,21 @@ Status ScanOperatorX::try_close(RuntimeState* state) { return Status::OK(); } -Status ScanLocalState::close(RuntimeState* state) { +template +Status ScanLocalState::close(RuntimeState* state) { if (_closed) { return Status::OK(); } if (_scanner_ctx.get()) { - _scanner_ctx->clear_and_join(reinterpret_cast(this), state); + _scanner_ctx->clear_and_join(reinterpret_cast(this), state); } - return PipelineXLocalState::close(state); + return PipelineXLocalState<>::close(state); } -Status ScanOperatorX::get_block(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { - auto& local_state = state->get_local_state(id())->cast(); +template +Status ScanOperatorX::get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) { + auto& local_state = state->get_local_state(id())->template cast(); SCOPED_TIMER(local_state._get_next_timer); SCOPED_TIMER(local_state.profile()->total_time_counter()); // in inverted index apply logic, in order to optimize query performance, @@ -1359,4 +1379,7 @@ Status ScanOperatorX::get_block(RuntimeState* state, vectorized::Block* block, return Status::OK(); } +template class ScanOperatorX; +template class ScanLocalState; + } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index f321e4e184e46cb..83589a5153fb7d1 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -23,6 +23,7 @@ #include "common/status.h" #include "operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/scan/vscan_node.h" namespace doris { @@ -54,38 +55,106 @@ class ScanOperator : public SourceOperator { Status try_close(RuntimeState* state) override; }; +class ScanLocalStateBase : public PipelineXLocalState<>, public vectorized::RuntimeFilterConsumer { +public: + ScanLocalStateBase(RuntimeState* state, OperatorXBase* parent) + : PipelineXLocalState<>(state, parent), + vectorized::RuntimeFilterConsumer(parent->id(), parent->runtime_filter_descs(), + parent->row_descriptor(), parent->conjuncts()) {} + virtual ~ScanLocalStateBase() = default; + + virtual bool ready_to_read() = 0; + + [[nodiscard]] virtual bool should_run_serial() const = 0; + + virtual RuntimeProfile* scanner_profile() = 0; + + [[nodiscard]] virtual const TupleDescriptor* input_tuple_desc() const = 0; + [[nodiscard]] virtual const TupleDescriptor* output_tuple_desc() const = 0; + + virtual int64_t limit_per_scanner() = 0; + + [[nodiscard]] virtual int runtime_filter_num() const = 0; + + Status virtual clone_conjunct_ctxs(vectorized::VExprContextSPtrs& conjuncts) = 0; + virtual void set_scan_ranges(const std::vector& scan_ranges) = 0; + + virtual TPushAggOp::type get_push_down_agg_type() = 0; + +protected: + friend class vectorized::ScannerContext; + friend class vectorized::VScanner; + + virtual Status _init_profile() = 0; + + std::shared_ptr _scanner_profile; + RuntimeProfile::Counter* _scanner_sched_counter = nullptr; + RuntimeProfile::Counter* _scanner_ctx_sched_counter = nullptr; + RuntimeProfile::Counter* _scanner_ctx_sched_time = nullptr; + RuntimeProfile::Counter* _scanner_wait_batch_timer = nullptr; + RuntimeProfile::Counter* _scanner_wait_worker_timer = nullptr; + // Num of newly created free blocks when running query + RuntimeProfile::Counter* _newly_create_free_blocks_num = nullptr; + // Max num of scanner thread + RuntimeProfile::Counter* _max_scanner_thread_num = nullptr; + // time of get block from scanner + RuntimeProfile::Counter* _scan_timer = nullptr; + RuntimeProfile::Counter* _scan_cpu_timer = nullptr; + // time of prefilter input block from scanner + RuntimeProfile::Counter* _prefilter_timer = nullptr; + // time of convert input block to output block from scanner + RuntimeProfile::Counter* _convert_block_timer = nullptr; + // time of filter output block from scanner + RuntimeProfile::Counter* _filter_timer = nullptr; + RuntimeProfile::Counter* _memory_usage_counter; + RuntimeProfile::HighWaterMarkCounter* _queued_blocks_memory_usage; + RuntimeProfile::HighWaterMarkCounter* _free_blocks_memory_usage; + // rows read from the scanner (including those discarded by (pre)filters) + RuntimeProfile::Counter* _rows_read_counter; + + // Wall based aggregate read throughput [rows/sec] + RuntimeProfile::Counter* _total_throughput_counter; + RuntimeProfile::Counter* _num_scanners; +}; + +template class ScanOperatorX; -class ScanLocalState : public PipelineXLocalState, public vectorized::RuntimeFilterConsumer { +template +class ScanLocalState : public ScanLocalStateBase { ENABLE_FACTORY_CREATOR(ScanLocalState); ScanLocalState(RuntimeState* state, OperatorXBase* parent); + virtual ~ScanLocalState() = default; Status init(RuntimeState* state, LocalStateInfo& info) override; Status close(RuntimeState* state) override; - bool ready_to_read(); + bool ready_to_read() override; - [[nodiscard]] bool should_run_serial() const; + [[nodiscard]] bool should_run_serial() const override; - RuntimeProfile* scanner_profile() { return _scanner_profile.get(); } + RuntimeProfile* scanner_profile() override { return _scanner_profile.get(); } - [[nodiscard]] const TupleDescriptor* input_tuple_desc() const; - [[nodiscard]] const TupleDescriptor* output_tuple_desc() const; + [[nodiscard]] const TupleDescriptor* input_tuple_desc() const override; + [[nodiscard]] const TupleDescriptor* output_tuple_desc() const override; - int64_t limit_per_scanner(); + int64_t limit_per_scanner() override; - [[nodiscard]] int runtime_filter_num() const { return (int)_runtime_filter_ctxs.size(); } + [[nodiscard]] int runtime_filter_num() const override { + return (int)_runtime_filter_ctxs.size(); + } - Status clone_conjunct_ctxs(vectorized::VExprContextSPtrs& conjuncts); - virtual void set_scan_ranges(const std::vector& scan_ranges) {} + Status clone_conjunct_ctxs(vectorized::VExprContextSPtrs& conjuncts) override; + virtual void set_scan_ranges(const std::vector& scan_ranges) override {} - TPushAggOp::type get_push_down_agg_type(); + TPushAggOp::type get_push_down_agg_type() override; protected: + template friend class ScanOperatorX; friend class vectorized::ScannerContext; friend class vectorized::VScanner; - virtual Status _init_profile(); + virtual Status _init_profile() override; virtual Status _process_conjuncts() { RETURN_IF_ERROR(_normalize_conjuncts()); return Status::OK(); @@ -259,50 +328,17 @@ class ScanLocalState : public PipelineXLocalState, public vectorized::RuntimeFil // "_colname_to_value_range" and in "_not_in_value_ranges" std::vector _not_in_value_ranges; - std::shared_ptr _scanner_profile; - - // rows read from the scanner (including those discarded by (pre)filters) - RuntimeProfile::Counter* _rows_read_counter; - // Wall based aggregate read throughput [rows/sec] - RuntimeProfile::Counter* _total_throughput_counter; - RuntimeProfile::Counter* _num_scanners; - RuntimeProfile::Counter* _get_next_timer = nullptr; RuntimeProfile::Counter* _open_timer = nullptr; RuntimeProfile::Counter* _alloc_resource_timer = nullptr; RuntimeProfile::Counter* _acquire_runtime_filter_timer = nullptr; - // time of get block from scanner - RuntimeProfile::Counter* _scan_timer = nullptr; - RuntimeProfile::Counter* _scan_cpu_timer = nullptr; - // time of prefilter input block from scanner - RuntimeProfile::Counter* _prefilter_timer = nullptr; - // time of convert input block to output block from scanner - RuntimeProfile::Counter* _convert_block_timer = nullptr; - // time of filter output block from scanner - RuntimeProfile::Counter* _filter_timer = nullptr; - - RuntimeProfile::Counter* _scanner_sched_counter = nullptr; - RuntimeProfile::Counter* _scanner_ctx_sched_counter = nullptr; - RuntimeProfile::Counter* _scanner_ctx_sched_time = nullptr; - RuntimeProfile::Counter* _scanner_wait_batch_timer = nullptr; - RuntimeProfile::Counter* _scanner_wait_worker_timer = nullptr; - // Num of newly created free blocks when running query - RuntimeProfile::Counter* _newly_create_free_blocks_num = nullptr; - // Max num of scanner thread - RuntimeProfile::Counter* _max_scanner_thread_num = nullptr; - - RuntimeProfile::Counter* _memory_usage_counter; - RuntimeProfile::HighWaterMarkCounter* _queued_blocks_memory_usage; - RuntimeProfile::HighWaterMarkCounter* _free_blocks_memory_usage; doris::Mutex _block_lock; }; -class ScanOperatorX : public OperatorXBase { +template +class ScanOperatorX : public OperatorX { public: - ScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string op_name); - // bool runtime_filters_are_ready_or_timeout() override; Status try_close(RuntimeState* state) override; @@ -317,11 +353,18 @@ class ScanOperatorX : public OperatorXBase { SourceState& source_state) override; bool is_source() const override { return true; } - const std::vector& runtime_filter_descs() { return _runtime_filter_descs; } + const std::vector& runtime_filter_descs() override { + return _runtime_filter_descs; + } TPushAggOp::type get_push_down_agg_type() { return _push_down_agg_type; } + using OperatorX::id; + protected: + ScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + virtual ~ScanOperatorX() = default; + template friend class ScanLocalState; friend class OlapScanLocalState; diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index af655b896cadce4..688c0ba07292581 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -29,12 +29,10 @@ namespace doris::pipeline { OPERATOR_CODE_GENERATOR(SortSinkOperator, StreamingOperator) Status SortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); auto& p = _parent->cast(); - _dependency = (SortDependency*)info.dependency; - _shared_state = (SortSharedState*)_dependency->shared_state(); RETURN_IF_ERROR(p._vsort_exec_exprs.clone(state, _vsort_exec_exprs)); - _profile = p._pool->add(new RuntimeProfile("SortSinkLocalState")); switch (p._algorithm) { case SortAlgorithm::HEAP_SORT: { _shared_state->sorter = vectorized::HeapSorter::create_unique( @@ -83,11 +81,10 @@ SortSinkOperatorX::SortSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, _limit(tnode.limit), _use_topn_opt(tnode.sort_node.use_topn_opt), _row_descriptor(descs, tnode.row_tuples, tnode.nullable_tuples), - _use_two_phase_read(tnode.sort_node.sort_info.use_two_phase_read) { - _name = "SortSinkOperatorX"; -} + _use_two_phase_read(tnode.sort_node.sort_info.use_two_phase_read) {} Status SortSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(DataSinkOperatorX::init(tnode, state)); RETURN_IF_ERROR(_vsort_exec_exprs.init(tnode.sort_node.sort_info, _pool)); _is_asc_order = tnode.sort_node.sort_info.is_asc_order; _nulls_first = tnode.sort_node.sort_info.nulls_first; @@ -144,12 +141,6 @@ Status SortSinkOperatorX::open(RuntimeState* state) { return _vsort_exec_exprs.open(state); } -Status SortSinkOperatorX::setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) { - auto local_state = SortSinkLocalState::create_shared(this, state); - state->emplace_sink_local_state(id(), local_state); - return local_state->init(state, info); -} - Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in_block, SourceState source_state) { auto& local_state = state->get_sink_local_state(id())->cast(); diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index c3150a87c48c5b9..b66977c115ace8e 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -20,6 +20,7 @@ #include #include "operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/core/field.h" #include "vec/exec/vsort_node.h" @@ -48,19 +49,17 @@ enum class SortAlgorithm { HEAP_SORT, TOPN_SORT, FULL_SORT }; class SortSinkOperatorX; -class SortSinkLocalState : public PipelineXSinkLocalState { +class SortSinkLocalState : public PipelineXSinkLocalState { ENABLE_FACTORY_CREATOR(SortSinkLocalState); public: - SortSinkLocalState(DataSinkOperatorX* parent, RuntimeState* state) - : PipelineXSinkLocalState(parent, state) {} + SortSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : PipelineXSinkLocalState(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; private: friend class SortSinkOperatorX; - SortDependency* _dependency; - SortSharedState* _shared_state; // Expressions and parameters used for build _sort_description vectorized::VSortExecExprs _vsort_exec_exprs; @@ -74,28 +73,23 @@ class SortSinkLocalState : public PipelineXSinkLocalState { vectorized::Field old_top {vectorized::Field::Types::Null}; }; -class SortSinkOperatorX final : public DataSinkOperatorX { +class SortSinkOperatorX final : public DataSinkOperatorX { public: SortSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); Status init(const TDataSink& tsink) override { - return Status::InternalError("{} should not init with TPlanNode", _name); + return Status::InternalError("{} should not init with TPlanNode", + DataSinkOperatorX::_name); } Status init(const TPlanNode& tnode, RuntimeState* state) override; Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; - Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) override; - Status sink(RuntimeState* state, vectorized::Block* in_block, SourceState source_state) override; bool can_write(RuntimeState* state) override { return true; } - void get_dependency(DependencySPtr& dependency) override { - dependency.reset(new SortDependency(id())); - } - private: friend class SortSinkLocalState; diff --git a/be/src/pipeline/exec/sort_source_operator.cpp b/be/src/pipeline/exec/sort_source_operator.cpp index 51a5d308e1c7af2..11fbb7686470955 100644 --- a/be/src/pipeline/exec/sort_source_operator.cpp +++ b/be/src/pipeline/exec/sort_source_operator.cpp @@ -26,19 +26,17 @@ namespace doris::pipeline { OPERATOR_CODE_GENERATOR(SortSourceOperator, SourceOperator) SortLocalState::SortLocalState(RuntimeState* state, OperatorXBase* parent) - : PipelineXLocalState(state, parent), _get_next_timer(nullptr) {} + : PipelineXLocalState(state, parent), _get_next_timer(nullptr) {} Status SortLocalState::init(RuntimeState* state, LocalStateInfo& info) { - RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); - _dependency = (SortDependency*)info.dependency; - _shared_state = (SortSharedState*)_dependency->shared_state(); + RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); _get_next_timer = ADD_TIMER(profile(), "GetResultTime"); return Status::OK(); } SortSourceOperatorX::SortSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs, std::string op_name) - : OperatorXBase(pool, tnode, descs, op_name) {} + const DescriptorTbl& descs) + : OperatorX(pool, tnode, descs) {} Status SortSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { @@ -61,18 +59,12 @@ bool SortSourceOperatorX::can_read(RuntimeState* state) { return local_state._dependency->done(); } -Status SortSourceOperatorX::setup_local_state(RuntimeState* state, LocalStateInfo& info) { - auto local_state = SortLocalState::create_shared(state, this); - state->emplace_local_state(id(), local_state); - return local_state->init(state, info); -} - Status SortLocalState::close(RuntimeState* state) { if (_closed) { return Status::OK(); } _shared_state->sorter = nullptr; - return PipelineXLocalState::close(state); + return PipelineXLocalState::close(state); } } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_source_operator.h b/be/src/pipeline/exec/sort_source_operator.h index 0cac457583364d9..9937343a16206e7 100644 --- a/be/src/pipeline/exec/sort_source_operator.h +++ b/be/src/pipeline/exec/sort_source_operator.h @@ -21,6 +21,7 @@ #include "common/status.h" #include "operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/vsort_node.h" namespace doris { @@ -45,7 +46,7 @@ class SortSourceOperator final : public SourceOperator { ENABLE_FACTORY_CREATOR(SortLocalState); public: @@ -57,20 +58,14 @@ class SortLocalState final : public PipelineXLocalState { private: friend class SortSourceOperatorX; - SortDependency* _dependency; - SortSharedState* _shared_state; - RuntimeProfile::Counter* _get_next_timer = nullptr; }; -class SortSourceOperatorX final : public OperatorXBase { +class SortSourceOperatorX final : public OperatorX { public: - SortSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string op_name); + SortSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); bool can_read(RuntimeState* state) override; - Status setup_local_state(RuntimeState* state, LocalStateInfo& info) override; - Status get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) override; diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp index 2b7184273e4d96d..6f71c36c41b2eb5 100644 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp @@ -135,9 +135,9 @@ static constexpr StreamingHtMinReductionEntry STREAMING_HT_MIN_REDUCTION[] = { static constexpr int STREAMING_HT_MIN_REDUCTION_SIZE = sizeof(STREAMING_HT_MIN_REDUCTION) / sizeof(STREAMING_HT_MIN_REDUCTION[0]); -StreamingAggSinkLocalState::StreamingAggSinkLocalState(DataSinkOperatorX* parent, +StreamingAggSinkLocalState::StreamingAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : AggSinkLocalState(parent, state), + : AggSinkLocalState(parent, state), _queue_byte_size_counter(nullptr), _queue_size_counter(nullptr), _streaming_agg_timer(nullptr) {} @@ -381,12 +381,6 @@ Status StreamingAggSinkOperatorX::sink(RuntimeState* state, vectorized::Block* i return Status::OK(); } -Status StreamingAggSinkOperatorX::setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) { - auto local_state = StreamingAggSinkLocalState::create_shared(this, state); - state->emplace_sink_local_state(id(), local_state); - return local_state->init(state, info); -} - Status StreamingAggSinkLocalState::close(RuntimeState* state) { if (_closed) { return Status::OK(); diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h index 190ab32228e1cdd..d3c5e1e14152c8a 100644 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h @@ -24,6 +24,7 @@ #include "aggregation_sink_operator.h" #include "common/status.h" #include "operator.h" +#include "pipeline/pipeline_x/operator.h" #include "util/runtime_profile.h" #include "vec/core/block.h" #include "vec/exec/vaggregation_node.h" @@ -72,11 +73,11 @@ class StreamingAggSinkOperator final : public StreamingOperator { public: - StreamingAggSinkLocalState(DataSinkOperatorX* parent, RuntimeState* state); + using Parent = StreamingAggSinkOperatorX; + ENABLE_FACTORY_CREATOR(StreamingAggSinkLocalState); + StreamingAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); Status init(RuntimeState* state, LocalSinkStateInfo& info) override; Status close(RuntimeState* state) override; @@ -102,20 +103,14 @@ class StreamingAggSinkLocalState final : public AggSinkLocalState { int64_t _num_rows_returned = 0; }; -class StreamingAggSinkOperatorX final : public AggSinkOperatorX { +class StreamingAggSinkOperatorX final : public AggSinkOperatorX { public: StreamingAggSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) override; - Status sink(RuntimeState* state, vectorized::Block* in_block, SourceState source_state) override; bool can_write(RuntimeState* state) override; - - void get_dependency(DependencySPtr& dependency) override { - dependency.reset(new AggDependency(id())); - } }; } // namespace pipeline -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp index c73dd26c19d1d03..6aac1130e106e14 100644 --- a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp @@ -73,9 +73,8 @@ OperatorPtr StreamingAggSourceOperatorBuilder::build_operator() { } StreamingAggSourceOperatorX::StreamingAggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs, - std::string op_name) - : AggSourceOperatorX(pool, tnode, descs, op_name) {} + const DescriptorTbl& descs) + : AggSourceOperatorX(pool, tnode, descs) {} Status StreamingAggSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.h b/be/src/pipeline/exec/streaming_aggregation_source_operator.h index b423e95b737863b..17c4058e59b718b 100644 --- a/be/src/pipeline/exec/streaming_aggregation_source_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.h @@ -23,6 +23,7 @@ #include "common/status.h" #include "operator.h" #include "pipeline/exec/aggregation_source_operator.h" +#include "pipeline/pipeline_x/operator.h" #include "vec/exec/vaggregation_node.h" namespace doris { @@ -62,7 +63,7 @@ class StreamingAggSourceOperator final : public SourceOperator); + // this operator in source open directly return, do this work in sink + Status open(RuntimeState* /*state*/) override { return Status::OK(); } + Status get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) override; bool can_read() override; diff --git a/be/src/pipeline/pipeline.cpp b/be/src/pipeline/pipeline.cpp index 69eaba3fbbc0702..ca5273f5837a28e 100644 --- a/be/src/pipeline/pipeline.cpp +++ b/be/src/pipeline/pipeline.cpp @@ -32,14 +32,14 @@ void Pipeline::_init_profile() { _pipeline_profile.reset(new RuntimeProfile(ss.str())); } -Status Pipeline::build_operators(Operators& operators) { +Status Pipeline::build_operators() { OperatorPtr pre; for (auto& operator_t : _operator_builders) { auto o = operator_t->build_operator(); if (pre) { o->set_child(pre); } - operators.emplace_back(o); + _operators.emplace_back(o); pre = std::move(o); } return Status::OK(); @@ -54,17 +54,17 @@ Status Pipeline::add_operator(OperatorBuilderPtr& op) { } Status Pipeline::add_operator(OperatorXPtr& op) { - if (_operators.empty() && !op->is_source()) { - return Status::InternalError("Should set source before other operator"); + operatorXs.emplace_back(op); + if (op->is_source()) { + std::reverse(operatorXs.begin(), operatorXs.end()); } - _operators.emplace_back(op); return Status::OK(); } Status Pipeline::prepare(RuntimeState* state) { // TODO - RETURN_IF_ERROR(_operators.back()->prepare(state)); - RETURN_IF_ERROR(_operators.back()->open(state)); + RETURN_IF_ERROR(operatorXs.back()->prepare(state)); + RETURN_IF_ERROR(operatorXs.back()->open(state)); RETURN_IF_ERROR(_sink_x->prepare(state)); RETURN_IF_ERROR(_sink_x->open(state)); return Status::OK(); diff --git a/be/src/pipeline/pipeline.h b/be/src/pipeline/pipeline.h index 114f51071d3e251..3c8c00098b3a8de 100644 --- a/be/src/pipeline/pipeline.h +++ b/be/src/pipeline/pipeline.h @@ -26,7 +26,8 @@ #include #include "common/status.h" -#include "exec/operator.h" +#include "pipeline/exec/operator.h" +#include "pipeline/pipeline_x/operator.h" #include "util/runtime_profile.h" namespace doris { @@ -45,31 +46,52 @@ using PipelineId = uint32_t; class Pipeline : public std::enable_shared_from_this { friend class PipelineTask; + friend class PipelineXTask; public: Pipeline() = delete; explicit Pipeline(PipelineId pipeline_id, std::weak_ptr context) - : _complete_dependency(0), _pipeline_id(pipeline_id), _context(context) { + : _pipeline_id(pipeline_id), _context(context) { _init_profile(); } void add_dependency(std::shared_ptr& pipeline) { - pipeline->_parents.push_back(weak_from_this()); - _dependencies.push_back(pipeline); + pipeline->_parents.push_back({_operator_builders.size(), weak_from_this()}); + _dependencies.push_back({_operator_builders.size(), pipeline}); } // If all dependencies are finished, this pipeline task should be scheduled. // e.g. Hash join probe task will be scheduled once Hash join build task is finished. - bool finish_one_dependency(int dependency_core_id) { - DCHECK(_complete_dependency < _dependencies.size()); - bool finish = _complete_dependency.fetch_add(1) == _dependencies.size() - 1; - if (finish) { + void finish_one_dependency(int dep_opr, int dependency_core_id) { + std::lock_guard l(_depend_mutex); + if (!_operators.empty() && _operators[dep_opr - 1]->can_terminate_early()) { + _always_can_read = true; + _always_can_write = (dep_opr == _operators.size()); + + for (int i = 0; i < _dependencies.size(); ++i) { + if (dep_opr == _dependencies[i].first) { + _dependencies.erase(_dependencies.begin(), _dependencies.begin() + i + 1); + break; + } + } + } else { + for (int i = 0; i < _dependencies.size(); ++i) { + if (dep_opr == _dependencies[i].first) { + _dependencies.erase(_dependencies.begin() + i); + break; + } + } + } + + if (_dependencies.empty()) { _previous_schedule_id = dependency_core_id; } - return finish; } - bool has_dependency() { return _complete_dependency.load() < _dependencies.size(); } + bool has_dependency() { + std::lock_guard l(_depend_mutex); + return !_dependencies.empty(); + } Status add_operator(OperatorBuilderPtr& op); @@ -82,29 +104,29 @@ class Pipeline : public std::enable_shared_from_this { Status set_sink(DataSinkOperatorXPtr& sink_operator); OperatorBuilderBase* sink() { return _sink.get(); } - DataSinkOperatorX* sink_x() { return _sink_x.get(); } - OperatorXs& operator_xs() { return _operators; } + DataSinkOperatorXBase* sink_x() { return _sink_x.get(); } + OperatorXs& operator_xs() { return operatorXs; } DataSinkOperatorXPtr sink_shared_pointer() { return _sink_x; } - Status build_operators(Operators&); + Status build_operators(); RuntimeProfile* pipeline_profile() { return _pipeline_profile.get(); } const RowDescriptor& output_row_desc() const { - return _operators[_operators.size() - 1]->row_desc(); + return operatorXs[operatorXs.size() - 1]->row_desc(); } PipelineId id() const { return _pipeline_id; } private: void _init_profile(); - std::atomic _complete_dependency; OperatorBuilders _operator_builders; // left is _source, right is _root OperatorBuilderPtr _sink; // put block to sink - std::vector> _parents; - std::vector> _dependencies; + std::mutex _depend_mutex; + std::vector>> _parents; + std::vector>> _dependencies; PipelineId _pipeline_id; std::weak_ptr _context; @@ -114,10 +136,38 @@ class Pipeline : public std::enable_shared_from_this { // Operators for pipelineX. All pipeline tasks share operators from this. // [SourceOperator -> ... -> SinkOperator] - OperatorXs _operators; + OperatorXs operatorXs; DataSinkOperatorXPtr _sink_x; std::shared_ptr _obj_pool; + + Operators _operators; + /** + * Consider the query plan below: + * + * ExchangeSource JoinBuild1 + * \ / + * JoinProbe1 (Right Outer) JoinBuild2 + * \ / + * JoinProbe2 (Right Outer) + * | + * Sink + * + * Assume JoinBuild1/JoinBuild2 outputs 0 rows, this pipeline task should not be blocked by ExchangeSource + * because we have a determined conclusion that JoinProbe1/JoinProbe2 will also output 0 rows. + * + * Assume JoinBuild2 outputs > 0 rows, this pipeline task may be blocked by Sink because JoinProbe2 will + * produce more data. + * + * Assume both JoinBuild2 outputs 0 rows this pipeline task should not be blocked by ExchangeSource + * and Sink because JoinProbe2 will always produce 0 rows and terminate early. + * + * In a nutshell, we should follow the rules: + * 1. if any operator in pipeline can terminate early, this task should never be blocked by source operator. + * 2. if the last operator (except sink) can terminate early, this task should never be blocked by sink operator. + */ + bool _always_can_read = false; + bool _always_can_write = false; }; } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index c2a5080eed8da5c..97689ed00144374 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -123,7 +123,6 @@ PipelineFragmentContext::PipelineFragmentContext( _fragment_id(fragment_id), _backend_num(backend_num), _exec_env(exec_env), - _cancel_reason(PPlanFragmentCancelReason::INTERNAL_ERROR), _query_ctx(std::move(query_ctx)), _call_back(call_back), _report_thread_active(false), @@ -137,48 +136,31 @@ PipelineFragmentContext::PipelineFragmentContext( } PipelineFragmentContext::~PipelineFragmentContext() { + auto st = _query_ctx->exec_status(); if (_runtime_state != nullptr) { // The memory released by the query end is recorded in the query mem tracker, main memory in _runtime_state. SCOPED_ATTACH_TASK(_runtime_state.get()); - _call_back(_runtime_state.get(), &_exec_status); + _call_back(_runtime_state.get(), &st); _runtime_state.reset(); } else { - _call_back(_runtime_state.get(), &_exec_status); + _call_back(_runtime_state.get(), &st); } DCHECK(!_report_thread_active); } void PipelineFragmentContext::cancel(const PPlanFragmentCancelReason& reason, const std::string& msg) { - if (!_runtime_state->is_cancelled()) { - std::lock_guard l(_status_lock); - if (_runtime_state->is_cancelled()) { - return; - } - if (reason != PPlanFragmentCancelReason::LIMIT_REACH) { - _exec_status = Status::Cancelled(msg); - } - _runtime_state->set_is_cancelled(true, msg); - - LOG(WARNING) << "PipelineFragmentContext Canceled. reason=" << msg; - - // Print detail informations below when you debugging here. - // - // for (auto& task : _tasks) { - // LOG(WARNING) << task->debug_string(); - // } + if (_query_ctx->cancel(true, msg, Status::Cancelled(msg))) { + LOG(WARNING) << "PipelineFragmentContext " + << PrintInstanceStandardInfo(_query_id, _fragment_id, _fragment_instance_id) + << " is canceled, cancel message: " << msg; - _runtime_state->set_process_status(_exec_status); // Get pipe from new load stream manager and send cancel to it or the fragment may hang to wait read from pipe // For stream load the fragment's query_id == load id, it is set in FE. auto stream_load_ctx = _exec_env->new_load_stream_mgr()->get(_query_id); if (stream_load_ctx != nullptr) { stream_load_ctx->pipe->cancel(msg); } - _cancel_reason = reason; - _cancel_msg = msg; - // To notify wait_for_start() - _query_ctx->set_ready_to_execute(true); // must close stream_mgr to avoid dead lock in Exchange Node _exec_env->vstream_mgr()->cancel(_fragment_instance_id); @@ -215,8 +197,8 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re } LOG_INFO("PipelineFragmentContext::prepare") - .tag("query_id", _query_id) - .tag("instance_id", local_params.fragment_instance_id) + .tag("query_id", print_id(_query_id)) + .tag("instance_id", print_id(local_params.fragment_instance_id)) .tag("backend_num", local_params.backend_num) .tag("pthread_id", (uintptr_t)pthread_self()); @@ -341,11 +323,9 @@ Status PipelineFragmentContext::_build_pipeline_tasks( // TODO pipeline 1 need to add new interface for exec node and operator sink->init(request.fragment.output_sink); - Operators operators; - RETURN_IF_ERROR(pipeline->build_operators(operators)); - auto task = - std::make_unique(pipeline, _total_tasks++, _runtime_state.get(), - operators, sink, this, pipeline->pipeline_profile()); + RETURN_IF_ERROR(pipeline->build_operators()); + auto task = std::make_unique(pipeline, _total_tasks++, _runtime_state.get(), + sink, this, pipeline->pipeline_profile()); sink->set_child(task->get_root()); _tasks.emplace_back(std::move(task)); _runtime_profile->add_child(pipeline->pipeline_profile(), true, nullptr); @@ -745,7 +725,7 @@ Status PipelineFragmentContext::_create_sink(int sender_id, const TDataSink& thr switch (thrift_sink.type) { case TDataSinkType::DATA_STREAM_SINK: { sink_ = std::make_shared(thrift_sink.stream_sink.dest_node_id, - _sink.get(), this); + _sink.get()); break; } case TDataSinkType::RESULT_SINK: { @@ -811,7 +791,7 @@ Status PipelineFragmentContext::_create_sink(int sender_id, const TDataSink& thr // 3. create and set sink operator of data stream sender for new pipeline OperatorBuilderPtr sink_op_builder = std::make_shared( - next_operator_builder_id(), _multi_cast_stream_sink_senders[i].get(), this, i); + next_operator_builder_id(), _multi_cast_stream_sink_senders[i].get(), i); new_pipeline->set_sink(sink_op_builder); // 4. init and prepare the data_stream_sender of diff exchange @@ -849,7 +829,7 @@ void PipelineFragmentContext::send_report(bool done) { Status exec_status = Status::OK(); { std::lock_guard l(_status_lock); - exec_status = _exec_status; + exec_status = _query_ctx->exec_status(); } // If plan is done successfully, but _is_report_success is false, diff --git a/be/src/pipeline/pipeline_fragment_context.h b/be/src/pipeline/pipeline_fragment_context.h index 6deb82156fad2ed..4b35c206e5ca5f4 100644 --- a/be/src/pipeline/pipeline_fragment_context.h +++ b/be/src/pipeline/pipeline_fragment_context.h @@ -101,6 +101,8 @@ class PipelineFragmentContext : public std::enable_shared_from_this l(_status_lock); - if (!status.ok() && _exec_status.ok()) { - _exec_status = status; + if (!status.ok() && _query_ctx->exec_status().ok()) { + _query_ctx->set_exec_status(status); } - return _exec_status; + return _query_ctx->exec_status(); } taskgroup::TaskGroupPipelineTaskEntity* get_task_group_entity() const { @@ -149,9 +151,6 @@ class PipelineFragmentContext : public std::enable_shared_from_this_operators), _source(_operators.front()), _root(_operators.back()), _sink(sink) { diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h index fc66ca54f139c9f..b7b730ad82a03fd 100644 --- a/be/src/pipeline/pipeline_task.h +++ b/be/src/pipeline/pipeline_task.h @@ -109,9 +109,8 @@ class PriorityTaskQueue; // The class do the pipeline task. Minest schdule union by task scheduler class PipelineTask { public: - PipelineTask(PipelinePtr& pipeline, uint32_t index, RuntimeState* state, Operators& operators, - OperatorPtr& sink, PipelineFragmentContext* fragment_context, - RuntimeProfile* parent_profile); + PipelineTask(PipelinePtr& pipeline, uint32_t index, RuntimeState* state, OperatorPtr& sink, + PipelineFragmentContext* fragment_context, RuntimeProfile* parent_profile); PipelineTask(PipelinePtr& pipeline, uint32_t index, RuntimeState* state, PipelineFragmentContext* fragment_context, RuntimeProfile* parent_profile); @@ -155,13 +154,13 @@ class PipelineTask { return false; } - virtual bool source_can_read() { return _source->can_read() || _ignore_blocking_source(); } + virtual bool source_can_read() { return _source->can_read() || _pipeline->_always_can_read; } virtual bool runtime_filters_are_ready_or_timeout() { return _source->runtime_filters_are_ready_or_timeout(); } - virtual bool sink_can_write() { return _sink->can_write() || _ignore_blocking_sink(); } + virtual bool sink_can_write() { return _sink->can_write() || _pipeline->_always_can_write; } virtual Status finalize(); @@ -251,7 +250,7 @@ class PipelineTask { protected: void _finish_p_dependency() { for (const auto& p : _pipeline->_parents) { - p.lock()->finish_one_dependency(_previous_schedule_id); + p.second.lock()->finish_one_dependency(p.first, _previous_schedule_id); } } @@ -348,44 +347,9 @@ class PipelineTask { RuntimeProfile::Counter* _pip_task_total_timer; private: - /** - * Consider the query plan below: - * - * ExchangeSource JoinBuild1 - * \ / - * JoinProbe1 (Right Outer) JoinBuild2 - * \ / - * JoinProbe2 (Right Outer) - * | - * Sink - * - * Assume JoinBuild1/JoinBuild2 outputs 0 rows, this pipeline task should not be blocked by ExchangeSource - * because we have a determined conclusion that JoinProbe1/JoinProbe2 will also output 0 rows. - * - * Assume JoinBuild2 outputs > 0 rows, this pipeline task may be blocked by Sink because JoinProbe2 will - * produce more data. - * - * Assume both JoinBuild2 outputs 0 rows this pipeline task should not be blocked by ExchangeSource - * and Sink because JoinProbe2 will always produce 0 rows and terminate early. - * - * In a nutshell, we should follow the rules: - * 1. if any operator in pipeline can terminate early, this task should never be blocked by source operator. - * 2. if the last operator (except sink) can terminate early, this task should never be blocked by sink operator. - */ - [[nodiscard]] bool _ignore_blocking_sink() { return _root->can_terminate_early(); } - - [[nodiscard]] bool _ignore_blocking_source() { - for (size_t i = 1; i < _operators.size(); i++) { - if (_operators[i]->can_terminate_early()) { - return true; - } - } - return false; - } - Operators _operators; // left is _source, right is _root OperatorPtr _source; OperatorPtr _root; OperatorPtr _sink; }; -} // namespace doris::pipeline \ No newline at end of file +} // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_x/dependency.cpp b/be/src/pipeline/pipeline_x/dependency.cpp index fab4db32f990322..d248ccb29bf4d8b 100644 --- a/be/src/pipeline/pipeline_x/dependency.cpp +++ b/be/src/pipeline/pipeline_x/dependency.cpp @@ -21,6 +21,18 @@ namespace doris::pipeline { +template Status JoinDependency::extract_join_column( + vectorized::Block&, + COW::mutable_ptr>&, + std::vector>&, + std::vector> const&); + +template Status JoinDependency::extract_join_column( + vectorized::Block&, + COW::mutable_ptr>&, + std::vector>&, + std::vector> const&); + Status AggDependency::reset_hash_table() { return std::visit( [&](auto&& agg_method) { @@ -217,4 +229,72 @@ bool AnalyticDependency::whether_need_next_partition(vectorized::BlockRowPos fou return false; } +Status JoinDependency::do_evaluate(vectorized::Block& block, vectorized::VExprContextSPtrs& exprs, + RuntimeProfile::Counter& expr_call_timer, + std::vector& res_col_ids) { + for (size_t i = 0; i < exprs.size(); ++i) { + int result_col_id = -1; + // execute build column + { + SCOPED_TIMER(&expr_call_timer); + RETURN_IF_ERROR(exprs[i]->execute(&block, &result_col_id)); + } + + // TODO: opt the column is const + block.get_by_position(result_col_id).column = + block.get_by_position(result_col_id).column->convert_to_full_column_if_const(); + res_col_ids[i] = result_col_id; + } + return Status::OK(); +} + +std::vector JoinDependency::convert_block_to_null(vectorized::Block& block) { + std::vector results; + for (int i = 0; i < block.columns(); ++i) { + if (auto& column_type = block.safe_get_by_position(i); !column_type.type->is_nullable()) { + DCHECK(!column_type.column->is_nullable()); + column_type.column = make_nullable(column_type.column); + column_type.type = make_nullable(column_type.type); + results.emplace_back(i); + } + } + return results; +} + +template +Status JoinDependency::extract_join_column(vectorized::Block& block, + vectorized::ColumnUInt8::MutablePtr& null_map, + vectorized::ColumnRawPtrs& raw_ptrs, + const std::vector& res_col_ids) { + for (size_t i = 0; i < _join_state.build_exprs_size; ++i) { + if (_join_state.is_null_safe_eq_join[i]) { + raw_ptrs[i] = block.get_by_position(res_col_ids[i]).column.get(); + } else { + auto column = block.get_by_position(res_col_ids[i]).column.get(); + if (auto* nullable = check_and_get_column(*column)) { + auto& col_nested = nullable->get_nested_column(); + auto& col_nullmap = nullable->get_null_map_data(); + + if constexpr (!BuildSide) { + DCHECK(null_map != nullptr); + vectorized::VectorizedUtils::update_null_map(null_map->get_data(), col_nullmap); + } + if (_join_state.store_null_in_hash_table[i]) { + raw_ptrs[i] = nullable; + } else { + if constexpr (BuildSide) { + DCHECK(null_map != nullptr); + vectorized::VectorizedUtils::update_null_map(null_map->get_data(), + col_nullmap); + } + raw_ptrs[i] = &col_nested; + } + } else { + raw_ptrs[i] = column; + } + } + } + return Status::OK(); +} + } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_x/dependency.h b/be/src/pipeline/pipeline_x/dependency.h index 152d0e69c1b8624..706f62657606dfd 100644 --- a/be/src/pipeline/pipeline_x/dependency.h +++ b/be/src/pipeline/pipeline_x/dependency.h @@ -19,6 +19,8 @@ #include "pipeline/exec/data_queue.h" #include "vec/common/sort/sorter.h" +#include "vec/exec/join/process_hash_table_probe.h" +#include "vec/exec/join/vhash_join_node.h" #include "vec/exec/vaggregation_node.h" #include "vec/exec/vanalytic_eval_node.h" @@ -43,6 +45,14 @@ class Dependency { std::atomic _done; }; +struct FakeSharedState {}; +struct FakeDependency : public Dependency { +public: + FakeDependency(int id) : Dependency(0) {} + using SharedState = FakeSharedState; + void* shared_state() override { return nullptr; } +}; + struct AggSharedState { public: AggSharedState() { @@ -71,6 +81,7 @@ struct AggSharedState { class AggDependency final : public Dependency { public: + using SharedState = AggSharedState; AggDependency(int id) : Dependency(id) { _mem_tracker = std::make_unique("AggregateOperator:"); } @@ -134,6 +145,7 @@ struct SortSharedState { class SortDependency final : public Dependency { public: + using SharedState = SortSharedState; SortDependency(int id) : Dependency(id) {} ~SortDependency() override = default; void* shared_state() override { return (void*)&_sort_state; }; @@ -167,6 +179,7 @@ struct AnalyticSharedState { class AnalyticDependency final : public Dependency { public: + using SharedState = AnalyticSharedState; AnalyticDependency(int id) : Dependency(id) {} ~AnalyticDependency() override = default; @@ -183,5 +196,52 @@ class AnalyticDependency final : public Dependency { AnalyticSharedState _analytic_state; }; +struct JoinSharedState { + // mark the join column whether support null eq + std::vector is_null_safe_eq_join; + // mark the build hash table whether it needs to store null value + std::vector store_null_in_hash_table; + // For some join case, we can apply a short circuit strategy + // 1. _short_circuit_for_null_in_probe_side = true + // 2. build side rows is empty, Join op is: inner join/right outer join/left semi/right semi/right anti + bool short_circuit_for_probe = false; + std::shared_ptr arena = std::make_shared(); + + // maybe share hash table with other fragment instances + std::shared_ptr hash_table_variants = + std::make_shared(); + vectorized::JoinOpVariants join_op_variants; + // for full/right outer join + vectorized::HashTableIteratorVariants outer_join_pull_visited_iter; + vectorized::HashTableIteratorVariants probe_row_match_iter; + std::shared_ptr> build_blocks; + vectorized::Sizes probe_key_sz; + const std::vector build_side_child_desc; + size_t build_exprs_size = 0; +}; + +class JoinDependency final : public Dependency { +public: + using SharedState = JoinSharedState; + JoinDependency(int id) : Dependency(id) {} + ~JoinDependency() override = default; + + void* shared_state() override { return (void*)&_join_state; } + + Status do_evaluate(vectorized::Block& block, vectorized::VExprContextSPtrs& exprs, + RuntimeProfile::Counter& expr_call_timer, std::vector& res_col_ids); + + std::vector convert_block_to_null(vectorized::Block& block); + + template + Status extract_join_column(vectorized::Block& block, + vectorized::ColumnUInt8::MutablePtr& null_map, + vectorized::ColumnRawPtrs& raw_ptrs, + const std::vector& res_col_ids); + +private: + JoinSharedState _join_state; +}; + } // namespace pipeline } // namespace doris diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp new file mode 100644 index 000000000000000..110acff33f689c6 --- /dev/null +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -0,0 +1,234 @@ +// 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 "operator.h" + +#include "pipeline/exec/aggregation_sink_operator.h" +#include "pipeline/exec/aggregation_source_operator.h" +#include "pipeline/exec/analytic_sink_operator.h" +#include "pipeline/exec/analytic_source_operator.h" +#include "pipeline/exec/exchange_sink_operator.h" +#include "pipeline/exec/exchange_source_operator.h" +#include "pipeline/exec/hashjoin_build_sink.h" +#include "pipeline/exec/hashjoin_probe_operator.h" +#include "pipeline/exec/olap_scan_operator.h" +#include "pipeline/exec/result_sink_operator.h" +#include "pipeline/exec/sort_sink_operator.h" +#include "pipeline/exec/sort_source_operator.h" +#include "pipeline/exec/streaming_aggregation_sink_operator.h" +#include "pipeline/exec/streaming_aggregation_source_operator.h" +#include "util/debug_util.h" + +namespace doris::pipeline { + +std::string OperatorXBase::debug_string() const { + std::stringstream ss; + ss << _op_name << ": is_source: " << is_source(); + ss << ", is_closed: " << _is_closed; + return ss.str(); +} + +Status OperatorXBase::init(const TPlanNode& tnode, RuntimeState* /*state*/) { + _op_name = print_plan_node_type(tnode.node_type) + "OperatorX"; + _init_runtime_profile(); + + if (tnode.__isset.vconjunct) { + vectorized::VExprContextSPtr context; + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(tnode.vconjunct, context)); + _conjuncts.emplace_back(context); + } else if (tnode.__isset.conjuncts) { + for (auto& conjunct : tnode.conjuncts) { + vectorized::VExprContextSPtr context; + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(conjunct, context)); + _conjuncts.emplace_back(context); + } + } + + // create the projections expr + if (tnode.__isset.projections) { + DCHECK(tnode.__isset.output_tuple_id); + RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(tnode.projections, _projections)); + } + return Status::OK(); +} + +Status OperatorXBase::prepare(RuntimeState* state) { + for (auto& conjunct : _conjuncts) { + RETURN_IF_ERROR(conjunct->prepare(state, intermediate_row_desc())); + } + + RETURN_IF_ERROR(vectorized::VExpr::prepare(_projections, state, intermediate_row_desc())); + + if (_child_x && !is_source()) { + RETURN_IF_ERROR(_child_x->prepare(state)); + } + + return Status::OK(); +} + +Status OperatorXBase::open(RuntimeState* state) { + for (auto& conjunct : _conjuncts) { + RETURN_IF_ERROR(conjunct->open(state)); + } + RETURN_IF_ERROR(vectorized::VExpr::open(_projections, state)); + if (_child_x && !is_source()) { + RETURN_IF_ERROR(_child_x->open(state)); + } + return Status::OK(); +} + +void OperatorXBase::_init_runtime_profile() { + std::stringstream ss; + ss << get_name() << " (id=" << _id << ")"; + _runtime_profile.reset(new RuntimeProfile(ss.str())); + _runtime_profile->set_metadata(_id); +} + +Status OperatorXBase::close(RuntimeState* state) { + if (_child_x && !is_source()) { + RETURN_IF_ERROR(_child_x->close(state)); + } + return state->get_local_state(id())->close(state); +} + +void PipelineXLocalStateBase::clear_origin_block() { + _origin_block.clear_column_data(_parent->_row_descriptor.num_materialized_slots()); +} + +Status OperatorXBase::do_projections(RuntimeState* state, vectorized::Block* origin_block, + vectorized::Block* output_block) { + auto local_state = state->get_local_state(id()); + SCOPED_TIMER(local_state->_projection_timer); + using namespace vectorized; + vectorized::MutableBlock mutable_block = + vectorized::VectorizedUtils::build_mutable_mem_reuse_block(output_block, + *_output_row_descriptor); + auto rows = origin_block->rows(); + + if (rows != 0) { + auto& mutable_columns = mutable_block.mutable_columns(); + DCHECK(mutable_columns.size() == _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)); + 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 + if (mutable_columns[i]->is_nullable() xor column_ptr->is_nullable()) { + DCHECK(mutable_columns[i]->is_nullable() && !column_ptr->is_nullable()); + reinterpret_cast(mutable_columns[i].get()) + ->insert_range_from_not_nullable(*column_ptr, 0, rows); + } else { + mutable_columns[i]->insert_range_from(*column_ptr, 0, rows); + } + } + DCHECK(mutable_block.rows() == rows); + } + + return Status::OK(); +} + +Status OperatorXBase::get_next_after_projects(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) { + auto local_state = state->get_local_state(id()); + if (_output_row_descriptor) { + local_state->clear_origin_block(); + auto status = get_block(state, &local_state->_origin_block, source_state); + if (UNLIKELY(!status.ok())) return status; + return do_projections(state, &local_state->_origin_block, block); + } + local_state->_peak_memory_usage_counter->set(local_state->_mem_tracker->peak_consumption()); + return get_block(state, block, source_state); +} + +bool PipelineXLocalStateBase::reached_limit() const { + return _parent->_limit != -1 && _num_rows_returned >= _parent->_limit; +} + +void PipelineXLocalStateBase::reached_limit(vectorized::Block* block, bool* eos) { + if (_parent->_limit != -1 and _num_rows_returned + block->rows() >= _parent->_limit) { + block->set_num_rows(_parent->_limit - _num_rows_returned); + *eos = true; + } + + _num_rows_returned += block->rows(); + COUNTER_SET(_rows_returned_counter, _num_rows_returned); +} + +void PipelineXLocalStateBase::reached_limit(vectorized::Block* block, SourceState& source_state) { + if (_parent->_limit != -1 and _num_rows_returned + block->rows() >= _parent->_limit) { + block->set_num_rows(_parent->_limit - _num_rows_returned); + source_state = SourceState::FINISHED; + } + + _num_rows_returned += block->rows(); + COUNTER_SET(_rows_returned_counter, _num_rows_returned); +} + +std::string DataSinkOperatorXBase::debug_string() const { + std::stringstream ss; + ss << _name << ", is_closed: " << _is_closed; + return ss.str(); +} + +Status DataSinkOperatorXBase::init(const TPlanNode& tnode, RuntimeState* state) { + _name = print_plan_node_type(tnode.node_type) + "SinkOperatorX"; + return Status::OK(); +} + +template +Status DataSinkOperatorX::setup_local_state(RuntimeState* state, + LocalSinkStateInfo& info) { + auto local_state = LocalStateType::create_shared(this, state); + state->emplace_sink_local_state(id(), local_state); + return local_state->init(state, info); +} + +template +void DataSinkOperatorX::get_dependency(DependencySPtr& dependency) { + dependency.reset(new typename LocalStateType::Dependency(id())); +} + +template +Status OperatorX::setup_local_state(RuntimeState* state, LocalStateInfo& info) { + auto local_state = LocalStateType::create_shared(state, this); + state->emplace_local_state(id(), local_state); + return local_state->init(state, info); +} + +#define DECLARE_OPERATOR_X(LOCAL_STATE) template class DataSinkOperatorX; +DECLARE_OPERATOR_X(HashJoinBuildSinkLocalState) +DECLARE_OPERATOR_X(ResultSinkLocalState) +DECLARE_OPERATOR_X(AnalyticSinkLocalState) +DECLARE_OPERATOR_X(SortSinkLocalState) +DECLARE_OPERATOR_X(BlockingAggSinkLocalState) +DECLARE_OPERATOR_X(StreamingAggSinkLocalState) +DECLARE_OPERATOR_X(ExchangeSinkLocalState) + +#undef DECLARE_OPERATOR_X + +#define DECLARE_OPERATOR_X(LOCAL_STATE) template class OperatorX; +DECLARE_OPERATOR_X(HashJoinProbeLocalState) +DECLARE_OPERATOR_X(OlapScanLocalState) +DECLARE_OPERATOR_X(AnalyticLocalState) +DECLARE_OPERATOR_X(SortLocalState) +DECLARE_OPERATOR_X(AggLocalState) +DECLARE_OPERATOR_X(ExchangeLocalState) + +#undef DECLARE_OPERATOR_X + +} // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h new file mode 100644 index 000000000000000..85cdeb7226ca8e9 --- /dev/null +++ b/be/src/pipeline/pipeline_x/operator.h @@ -0,0 +1,492 @@ +// 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 "pipeline/exec/operator.h" + +namespace doris::pipeline { + +// This struct is used only for initializing local state. +struct LocalStateInfo { + const std::vector scan_ranges; + Dependency* dependency; + std::shared_ptr recvr; +}; + +// This struct is used only for initializing local sink state. +struct LocalSinkStateInfo { + const int sender_id; + Dependency* dependency; + std::shared_ptr sender; +}; + +class PipelineXLocalStateBase { +public: + PipelineXLocalStateBase(RuntimeState* state, OperatorXBase* parent) + : _num_rows_returned(0), + _rows_returned_counter(nullptr), + _rows_returned_rate(nullptr), + _memory_used_counter(nullptr), + _peak_memory_usage_counter(nullptr), + _parent(parent), + _state(state) {} + virtual ~PipelineXLocalStateBase() = default; + + template + TARGET& cast() { + return reinterpret_cast(*this); + } + template + const TARGET& cast() const { + return reinterpret_cast(*this); + } + + virtual Status init(RuntimeState* state, LocalStateInfo& info) = 0; + virtual Status close(RuntimeState* state) = 0; + + // If use projection, we should clear `_origin_block`. + void clear_origin_block(); + + bool reached_limit() const; + void reached_limit(vectorized::Block* block, bool* eos); + void reached_limit(vectorized::Block* block, SourceState& source_state); + RuntimeProfile* profile() { return _runtime_profile.get(); } + + MemTracker* mem_tracker() { return _mem_tracker.get(); } + RuntimeProfile::Counter* rows_returned_counter() { return _rows_returned_counter; } + RuntimeProfile::Counter* rows_returned_rate() { return _rows_returned_rate; } + RuntimeProfile::Counter* memory_used_counter() { return _memory_used_counter; } + RuntimeProfile::Counter* projection_timer() { return _projection_timer; } + + OperatorXBase* parent() { return _parent; } + RuntimeState* state() { return _state; } + vectorized::VExprContextSPtrs& conjuncts() { return _conjuncts; } + vectorized::VExprContextSPtrs& projections() { return _projections; } + int64_t num_rows_returned() const { return _num_rows_returned; } + void add_num_rows_returned(int64_t delta) { _num_rows_returned += delta; } + void set_num_rows_returned(int64_t value) { _num_rows_returned = value; } + +protected: + friend class OperatorXBase; + + ObjectPool* _pool; + int64_t _num_rows_returned; + + std::unique_ptr _runtime_profile; + + // Record this node memory size. it is expected that artificial guarantees are accurate, + // which will providea reference for operator memory. + std::unique_ptr _mem_tracker; + + RuntimeProfile::Counter* _rows_returned_counter; + RuntimeProfile::Counter* _rows_returned_rate; + // Account for peak memory used by this node + RuntimeProfile::Counter* _memory_used_counter; + RuntimeProfile::Counter* _projection_timer; + // Account for peak memory used by this node + RuntimeProfile::Counter* _peak_memory_usage_counter; + + OpentelemetrySpan _span; + OperatorXBase* _parent; + RuntimeState* _state; + vectorized::VExprContextSPtrs _conjuncts; + vectorized::VExprContextSPtrs _projections; + bool _closed = false; + vectorized::Block _origin_block; +}; + +class OperatorXBase : public OperatorBase { +public: + OperatorXBase(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) + : OperatorBase(nullptr), + _id(tnode.node_id), + _type(tnode.node_type), + _pool(pool), + _tuple_ids(tnode.row_tuples), + _row_descriptor(descs, tnode.row_tuples, tnode.nullable_tuples), + _resource_profile(tnode.resource_profile), + _limit(tnode.limit) { + if (tnode.__isset.output_tuple_id) { + _output_row_descriptor.reset(new RowDescriptor(descs, {tnode.output_tuple_id}, {true})); + } + } + + virtual Status init(const TPlanNode& tnode, RuntimeState* state); + Status init(const TDataSink& tsink) override { + LOG(FATAL) << "should not reach here!"; + return Status::OK(); + } + [[nodiscard]] RuntimeProfile* get_runtime_profile() const override { + return _runtime_profile.get(); + } + [[noreturn]] virtual const std::vector& runtime_filter_descs() { + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, _op_name); + } + [[nodiscard]] std::string get_name() const override { return _op_name; } + + virtual Status prepare(RuntimeState* state) override; + + virtual Status open(RuntimeState* state) override; + + Status finalize(RuntimeState* state) override { return Status::OK(); } + + [[nodiscard]] bool can_terminate_early() override { return false; } + + [[nodiscard]] virtual bool can_terminate_early(RuntimeState* state) { return false; } + + bool can_read() override { + LOG(FATAL) << "should not reach here!"; + return false; + } + + bool can_write() override { + LOG(FATAL) << "should not reach here!"; + return false; + } + + bool is_pending_finish() const override { + LOG(FATAL) << "should not reach here!"; + return false; + } + + Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override { + LOG(FATAL) << "should not reach here!"; + return Status::OK(); + } + + virtual Status close(RuntimeState* state) override; + + virtual bool can_read(RuntimeState* state) { return true; } + + virtual bool is_pending_finish(RuntimeState* state) const { return false; } + + [[nodiscard]] virtual const RowDescriptor& intermediate_row_desc() const { + return _row_descriptor; + } + + virtual std::string debug_string() const override; + + virtual Status setup_local_state(RuntimeState* state, LocalStateInfo& info) = 0; + + template + TARGET& cast() { + DCHECK(dynamic_cast(this)) + << " Mismatch type! Current type is " << typeid(*this).name() + << " and expect type is" << typeid(TARGET).name(); + return reinterpret_cast(*this); + } + template + const TARGET& cast() const { + DCHECK(dynamic_cast(this)) + << " Mismatch type! Current type is " << typeid(*this).name() + << " and expect type is" << typeid(TARGET).name(); + return reinterpret_cast(*this); + } + + [[nodiscard]] OperatorXPtr get_child() { return _child_x; } + + [[nodiscard]] vectorized::VExprContextSPtrs& conjuncts() { return _conjuncts; } + [[nodiscard]] RowDescriptor& row_descriptor() { return _row_descriptor; } + + [[nodiscard]] int id() const override { return _id; } + + [[nodiscard]] int64_t limit() const { return _limit; } + + [[nodiscard]] virtual const RowDescriptor& row_desc() override { + return _output_row_descriptor ? *_output_row_descriptor : _row_descriptor; + } + + [[nodiscard]] virtual bool is_source() const override { return false; } + [[nodiscard]] virtual bool need_to_create_exch_recv() const { return false; } + + Status get_next_after_projects(RuntimeState* state, vectorized::Block* block, + SourceState& source_state); + + /// Only use in vectorized exec engine try to do projections to trans _row_desc -> _output_row_desc + Status do_projections(RuntimeState* state, vectorized::Block* origin_block, + vectorized::Block* output_block); + +protected: + template + friend class PipelineXLocalState; + friend class PipelineXLocalStateBase; + int _id; // unique w/in single plan tree + TPlanNodeType::type _type; + ObjectPool* _pool; + std::vector _tuple_ids; + + vectorized::VExprContextSPtrs _conjuncts; + + RowDescriptor _row_descriptor; + + std::unique_ptr _output_row_descriptor; + vectorized::VExprContextSPtrs _projections; + + /// Resource information sent from the frontend. + const TBackendResourceProfile _resource_profile; + + int64_t _limit; // -1: no limit + std::unique_ptr _runtime_profile; + +private: + void _init_runtime_profile(); + + std::string _op_name; +}; + +template +class OperatorX : public OperatorXBase { +public: + OperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) + : OperatorXBase(pool, tnode, descs) {} + virtual ~OperatorX() = default; + + Status setup_local_state(RuntimeState* state, LocalStateInfo& info) override; +}; + +template +class PipelineXLocalState : public PipelineXLocalStateBase { +public: + PipelineXLocalState(RuntimeState* state, OperatorXBase* parent) + : PipelineXLocalStateBase(state, parent) {} + virtual ~PipelineXLocalState() {} + + virtual Status init(RuntimeState* state, LocalStateInfo& info) override { + _dependency = (DependencyType*)info.dependency; + if (_dependency) { + _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state(); + } + _runtime_profile.reset(new RuntimeProfile("LocalState " + _parent->get_name())); + _parent->get_runtime_profile()->add_child(_runtime_profile.get(), true, nullptr); + _conjuncts.resize(_parent->_conjuncts.size()); + _projections.resize(_parent->_projections.size()); + for (size_t i = 0; i < _conjuncts.size(); i++) { + RETURN_IF_ERROR(_parent->_conjuncts[i]->clone(state, _conjuncts[i])); + } + for (size_t i = 0; i < _projections.size(); i++) { + RETURN_IF_ERROR(_parent->_projections[i]->clone(state, _projections[i])); + } + DCHECK(_runtime_profile.get() != nullptr); + _rows_returned_counter = ADD_COUNTER(_runtime_profile, "RowsReturned", TUnit::UNIT); + _projection_timer = ADD_TIMER(_runtime_profile, "ProjectionTime"); + _rows_returned_rate = profile()->add_derived_counter( + doris::ExecNode::ROW_THROUGHPUT_COUNTER, TUnit::UNIT_PER_SECOND, + std::bind(&RuntimeProfile::units_per_second, _rows_returned_counter, + profile()->total_time_counter()), + ""); + _mem_tracker = + std::make_unique("PipelineXLocalState:" + _runtime_profile->name()); + _memory_used_counter = ADD_LABEL_COUNTER(_runtime_profile, "MemoryUsage"); + _peak_memory_usage_counter = _runtime_profile->AddHighWaterMarkCounter( + "PeakMemoryUsage", TUnit::BYTES, "MemoryUsage"); + return Status::OK(); + } + virtual Status close(RuntimeState* state) override { + if (_closed) { + return Status::OK(); + } + if (_rows_returned_counter != nullptr) { + COUNTER_SET(_rows_returned_counter, _num_rows_returned); + } + profile()->add_to_span(_span); + _closed = true; + return Status::OK(); + } + +protected: + DependencyType* _dependency; + typename DependencyType::SharedState* _shared_state; +}; + +class DataSinkOperatorXBase; + +class PipelineXSinkLocalStateBase { +public: + PipelineXSinkLocalStateBase(DataSinkOperatorXBase* parent_, RuntimeState* state_) + : _parent(parent_), _state(state_) {} + virtual ~PipelineXSinkLocalStateBase() {} + + virtual Status init(RuntimeState* state, LocalSinkStateInfo& info) = 0; + virtual Status close(RuntimeState* state) = 0; + template + TARGET& cast() { + DCHECK(dynamic_cast(this)) + << " Mismatch type! Current type is " << typeid(*this).name() + << " and expect type is" << typeid(TARGET).name(); + return reinterpret_cast(*this); + } + template + const TARGET& cast() const { + DCHECK(dynamic_cast(this)) + << " Mismatch type! Current type is " << typeid(*this).name() + << " and expect type is" << typeid(TARGET).name(); + return reinterpret_cast(*this); + } + + DataSinkOperatorXBase* parent() { return _parent; } + RuntimeState* state() { return _state; } + RuntimeProfile* profile() { return _profile; } + MemTracker* mem_tracker() { return _mem_tracker.get(); } + QueryStatistics* query_statistics() { return _query_statistics.get(); } + RuntimeProfile* faker_runtime_profile() const { return _faker_runtime_profile.get(); } + +protected: + DataSinkOperatorXBase* _parent; + RuntimeState* _state; + RuntimeProfile* _profile; + std::unique_ptr _mem_tracker; + // Maybe this will be transferred to BufferControlBlock. + std::shared_ptr _query_statistics; + // Set to true after close() has been called. subclasses should check and set this in + // close(). + bool _closed = false; + //NOTICE: now add a faker profile, because sometimes the profile record is useless + //so we want remove some counters and timers, eg: in join node, if it's broadcast_join + //and shared hash table, some counter/timer about build hash table is useless, + //so we could add those counter/timer in faker profile, and those will not display in web profile. + std::unique_ptr _faker_runtime_profile = + std::make_unique("faker profile"); +}; + +class DataSinkOperatorXBase : public OperatorBase { +public: + DataSinkOperatorXBase(const int id) : OperatorBase(nullptr), _id(id) {} + + virtual ~DataSinkOperatorXBase() override = default; + + // For agg/sort/join sink. + virtual Status init(const TPlanNode& tnode, RuntimeState* state); + + virtual Status init(const TDataSink& tsink) override { return Status::OK(); } + + virtual Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) = 0; + + [[nodiscard]] virtual bool need_to_create_result_sender() const { return false; } + + template + TARGET& cast() { + DCHECK(dynamic_cast(this)) + << " Mismatch type! Current type is " << typeid(*this).name() + << " and expect type is" << typeid(TARGET).name(); + return reinterpret_cast(*this); + } + template + const TARGET& cast() const { + DCHECK(dynamic_cast(this)) + << " Mismatch type! Current type is " << typeid(*this).name() + << " and expect type is" << typeid(TARGET).name(); + return reinterpret_cast(*this); + } + + virtual void get_dependency(DependencySPtr& dependency) = 0; + + virtual Status close(RuntimeState* state) override { + return state->get_sink_local_state(id())->close(state); + } + + bool can_read() override { + LOG(FATAL) << "should not reach here!"; + return false; + } + + bool can_write() override { + LOG(FATAL) << "should not reach here!"; + return false; + } + + bool is_pending_finish() const override { + LOG(FATAL) << "should not reach here!"; + return false; + } + + virtual bool can_write(RuntimeState* state) { return false; } + + virtual bool is_pending_finish(RuntimeState* state) const { return false; } + + virtual std::string debug_string() const override; + + bool is_sink() const override { return true; } + + bool is_source() const override { return false; } + + virtual Status close(RuntimeState* state, Status exec_status) { return Status::OK(); } + + [[nodiscard]] RuntimeProfile* get_runtime_profile() const override { return _profile; } + + [[nodiscard]] int id() const override { return _id; } + + [[nodiscard]] std::string get_name() const override { return _name; } + + Status finalize(RuntimeState* state) override { return Status::OK(); } + +protected: + const int _id; + std::string _name; + + // Maybe this will be transferred to BufferControlBlock. + std::shared_ptr _query_statistics; + + OpentelemetrySpan _span {}; + + RuntimeProfile* _profile = nullptr; +}; + +template +class DataSinkOperatorX : public DataSinkOperatorXBase { +public: + DataSinkOperatorX(const int id) : DataSinkOperatorXBase(id) {} + + virtual ~DataSinkOperatorX() override = default; + + Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) override; + + void get_dependency(DependencySPtr& dependency) override; +}; + +template +class PipelineXSinkLocalState : public PipelineXSinkLocalStateBase { +public: + using Dependency = DependencyType; + PipelineXSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : PipelineXSinkLocalStateBase(parent, state) {} + virtual ~PipelineXSinkLocalState() {} + + virtual Status init(RuntimeState* state, LocalSinkStateInfo& info) override { + _dependency = (DependencyType*)info.dependency; + if (_dependency) { + _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state(); + } + // create profile + _profile = state->obj_pool()->add(new RuntimeProfile(_parent->get_name())); + _mem_tracker = std::make_unique(_parent->get_name()); + return Status::OK(); + } + virtual Status close(RuntimeState* state) override { + if (_closed) { + return Status::OK(); + } + _closed = true; + return Status::OK(); + } + +protected: + DependencyType* _dependency; + typename DependencyType::SharedState* _shared_state; +}; + +} // namespace doris::pipeline 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 1a569202fbfdd1b..cdb8e689f596925 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -49,6 +49,8 @@ #include "pipeline/exec/datagen_operator.h" #include "pipeline/exec/exchange_sink_operator.h" #include "pipeline/exec/exchange_source_operator.h" +#include "pipeline/exec/hashjoin_build_sink.h" +#include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/olap_scan_operator.h" #include "pipeline/exec/result_sink_operator.h" #include "pipeline/exec/scan_operator.h" @@ -97,13 +99,13 @@ PipelineXFragmentContext::PipelineXFragmentContext( call_back, report_status_cb) {} PipelineXFragmentContext::~PipelineXFragmentContext() { + auto st = _query_ctx->exec_status(); if (!_runtime_states.empty()) { // The memory released by the query end is recorded in the query mem tracker, main memory in _runtime_state. SCOPED_ATTACH_TASK(_runtime_state.get()); - FOR_EACH_RUNTIME_STATE(_call_back(runtime_state.get(), &_exec_status); - runtime_state.reset();) + FOR_EACH_RUNTIME_STATE(_call_back(runtime_state.get(), &st); runtime_state.reset();) } else { - _call_back(nullptr, &_exec_status); + _call_back(nullptr, &st); } _runtime_state.reset(); DCHECK(!_report_thread_active); @@ -111,35 +113,21 @@ PipelineXFragmentContext::~PipelineXFragmentContext() { void PipelineXFragmentContext::cancel(const PPlanFragmentCancelReason& reason, const std::string& msg) { - if (!_runtime_state->is_cancelled()) { - std::lock_guard l(_status_lock); - if (_runtime_state->is_cancelled()) { - return; - } - if (reason != PPlanFragmentCancelReason::LIMIT_REACH) { - _exec_status = Status::Cancelled(msg); - } - - FOR_EACH_RUNTIME_STATE( - runtime_state->set_is_cancelled(true, msg); - runtime_state->set_process_status(_exec_status); - _exec_env->vstream_mgr()->cancel(runtime_state->fragment_instance_id());) - + if (_query_ctx->cancel(true, msg, Status::Cancelled(msg))) { LOG(WARNING) << "PipelineFragmentContext Canceled. reason=" << msg; - // Get pipe from new load stream manager and send cancel to it or the fragment may hang to wait read from pipe // For stream load the fragment's query_id == load id, it is set in FE. auto stream_load_ctx = _exec_env->new_load_stream_mgr()->get(_query_id); if (stream_load_ctx != nullptr) { stream_load_ctx->pipe->cancel(msg); } - _cancel_reason = reason; - _cancel_msg = msg; - // To notify wait_for_start() - _query_ctx->set_ready_to_execute(true); // 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)); @@ -214,6 +202,12 @@ Status PipelineXFragmentContext::prepare(const doris::TPipelineFragmentParams& r // 4. Initialize global states in pipelines. for (PipelinePtr& pipeline : _pipelines) { + for (auto& op : pipeline->operator_xs()) { + if (op->get_child() && !op->is_source()) { + op->get_runtime_profile()->add_child(op->get_child()->get_runtime_profile(), true, + nullptr); + } + } pipeline->sink_x()->set_child(pipeline->operator_xs().back()); RETURN_IF_ERROR(pipeline->prepare(_runtime_state.get())); } @@ -244,7 +238,7 @@ Status PipelineXFragmentContext::_create_data_sink(ObjectPool* pool, const TData : false; _sink.reset(new ExchangeSinkOperatorX(state, pool, row_desc, thrift_sink.stream_sink, params.destinations, - send_query_statistics_with_every_batch, this)); + send_query_statistics_with_every_batch)); break; } case TDataSinkType::RESULT_SINK: { @@ -363,6 +357,8 @@ Status PipelineXFragmentContext::_build_pipeline_tasks( } } } + _build_side_pipelines.clear(); + _dag.clear(); // register the profile of child data stream sender // for (auto& sender : _multi_cast_stream_sink_senders) { // _sink->profile()->add_child(sender->profile(), true, nullptr); @@ -439,7 +435,7 @@ Status PipelineXFragmentContext::_build_pipelines(ObjectPool* pool, int node_idx = 0; RETURN_IF_ERROR(_create_tree_helper(pool, request.fragment.plan.nodes, request, descs, nullptr, - &node_idx, root, cur_pipe)); + &node_idx, root, cur_pipe, 0)); if (node_idx + 1 != request.fragment.plan.nodes.size()) { // TODO: print thrift msg for diagnostic purposes. @@ -455,7 +451,8 @@ Status PipelineXFragmentContext::_create_tree_helper(ObjectPool* pool, const doris::TPipelineFragmentParams& request, const DescriptorTbl& descs, OperatorXPtr parent, int* node_idx, - OperatorXPtr* root, PipelinePtr& cur_pipe) { + OperatorXPtr* root, PipelinePtr& cur_pipe, + int child_idx) { // propagate error case if (*node_idx >= tnodes.size()) { // TODO: print thrift msg @@ -467,7 +464,8 @@ Status PipelineXFragmentContext::_create_tree_helper(ObjectPool* pool, int num_children = tnodes[*node_idx].num_children; OperatorXPtr op = nullptr; - RETURN_IF_ERROR(_create_operator(pool, tnodes[*node_idx], request, descs, op, cur_pipe)); + RETURN_IF_ERROR(_create_operator(pool, tnodes[*node_idx], request, descs, op, cur_pipe, + parent == nullptr ? -1 : parent->id(), child_idx)); // assert(parent != nullptr || (node_idx == 0 && root_expr != nullptr)); if (parent != nullptr) { @@ -478,8 +476,8 @@ Status PipelineXFragmentContext::_create_tree_helper(ObjectPool* pool, for (int i = 0; i < num_children; i++) { ++*node_idx; - RETURN_IF_ERROR( - _create_tree_helper(pool, tnodes, request, descs, op, node_idx, nullptr, cur_pipe)); + RETURN_IF_ERROR(_create_tree_helper(pool, tnodes, request, descs, op, node_idx, nullptr, + cur_pipe, i)); // we are expecting a child, but have used all nodes // this means we have been given a bad tree and must fail @@ -493,38 +491,35 @@ Status PipelineXFragmentContext::_create_tree_helper(ObjectPool* pool, RETURN_IF_ERROR(op->init(tnode, _runtime_state.get())); - if (op->get_child() && !op->is_source()) { - op->get_runtime_profile()->add_child(op->get_child()->get_runtime_profile(), true, nullptr); - } - return Status::OK(); } Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanNode& tnode, const doris::TPipelineFragmentParams& request, const DescriptorTbl& descs, OperatorXPtr& op, - PipelinePtr& cur_pipe) { + PipelinePtr& cur_pipe, int parent_idx, + int child_idx) { + if (_build_side_pipelines.find(parent_idx) != _build_side_pipelines.end() && child_idx > 0) { + cur_pipe = _build_side_pipelines[parent_idx]; + } std::stringstream error_msg; - switch (tnode.node_type) { case TPlanNodeType::OLAP_SCAN_NODE: { - op.reset(new OlapScanOperatorX(pool, tnode, descs, "ScanOperatorX")); + op.reset(new OlapScanOperatorX(pool, tnode, descs)); RETURN_IF_ERROR(cur_pipe->add_operator(op)); break; } case TPlanNodeType::EXCHANGE_NODE: { int num_senders = find_with_default(request.per_exch_num_senders, tnode.node_id, 0); DCHECK_GT(num_senders, 0); - op.reset(new ExchangeSourceOperatorX(pool, tnode, descs, "ExchangeSourceXOperator", - num_senders)); + op.reset(new ExchangeSourceOperatorX(pool, tnode, descs, num_senders)); RETURN_IF_ERROR(cur_pipe->add_operator(op)); break; } case TPlanNodeType::AGGREGATION_NODE: { if (tnode.agg_node.__isset.use_streaming_preaggregation && tnode.agg_node.use_streaming_preaggregation) { - op.reset(new StreamingAggSourceOperatorX(pool, tnode, descs, - "StreamingAggSourceXOperator")); + op.reset(new StreamingAggSourceOperatorX(pool, tnode, descs)); RETURN_IF_ERROR(cur_pipe->add_operator(op)); const auto downstream_pipeline_id = cur_pipe->id(); @@ -538,7 +533,7 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN RETURN_IF_ERROR(cur_pipe->set_sink(sink)); RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); } else { - op.reset(new AggSourceOperatorX(pool, tnode, descs, "AggSourceXOperator")); + op.reset(new AggSourceOperatorX(pool, tnode, descs)); RETURN_IF_ERROR(cur_pipe->add_operator(op)); const auto downstream_pipeline_id = cur_pipe->id(); @@ -555,8 +550,26 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN } break; } + case TPlanNodeType::HASH_JOIN_NODE: { + op.reset(new HashJoinProbeOperatorX(pool, tnode, descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + PipelinePtr build_side_pipe = add_pipeline(); + _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); + + DataSinkOperatorXPtr sink; + sink.reset(new HashJoinBuildSinkOperatorX(pool, tnode, descs)); + RETURN_IF_ERROR(build_side_pipe->set_sink(sink)); + RETURN_IF_ERROR(build_side_pipe->sink_x()->init(tnode, _runtime_state.get())); + _build_side_pipelines.insert({sink->id(), build_side_pipe}); + break; + } case TPlanNodeType::SORT_NODE: { - op.reset(new SortSourceOperatorX(pool, tnode, descs, "SortSourceXOperator")); + op.reset(new SortSourceOperatorX(pool, tnode, descs)); RETURN_IF_ERROR(cur_pipe->add_operator(op)); const auto downstream_pipeline_id = cur_pipe->id(); @@ -573,7 +586,7 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN break; } case TPlanNodeType::ANALYTIC_EVAL_NODE: { - op.reset(new AnalyticSourceOperatorX(pool, tnode, descs, "AnalyticSourceXOperator")); + op.reset(new AnalyticSourceOperatorX(pool, tnode, descs)); RETURN_IF_ERROR(cur_pipe->add_operator(op)); const auto downstream_pipeline_id = cur_pipe->id(); @@ -665,7 +678,7 @@ void PipelineXFragmentContext::send_report(bool done) { Status exec_status = Status::OK(); { std::lock_guard l(_status_lock); - exec_status = _exec_status; + exec_status = _query_ctx->exec_status(); } // If plan is done successfully, but _is_report_success is false, diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h index 075b2bc93180362..af32f5e705e4b22 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h @@ -103,11 +103,12 @@ class PipelineXFragmentContext : public PipelineFragmentContext { Status _create_tree_helper(ObjectPool* pool, const std::vector& tnodes, const doris::TPipelineFragmentParams& request, const DescriptorTbl& descs, OperatorXPtr parent, int* node_idx, - OperatorXPtr* root, PipelinePtr& cur_pipe); + OperatorXPtr* root, PipelinePtr& cur_pipe, int child_idx); Status _create_operator(ObjectPool* pool, const TPlanNode& tnode, const doris::TPipelineFragmentParams& request, - const DescriptorTbl& descs, OperatorXPtr& node, PipelinePtr& cur_pipe); + const DescriptorTbl& descs, OperatorXPtr& node, PipelinePtr& cur_pipe, + int parent_idx, int child_idx); Status _create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, const std::vector& output_exprs, @@ -128,6 +129,12 @@ class PipelineXFragmentContext : public PipelineFragmentContext { // `_dag` manage dependencies between pipelines by pipeline ID std::map> _dag; + + // We use preorder traversal to create an operator tree. When we meet a join node, we should + // build probe operator and build operator in separate pipelines. To do this, we should build + // ProbeSide first, and use `_pipelines_to_build` to store which pipeline the build operator + // is in, so we can build BuildSide once we complete probe side. + std::map _build_side_pipelines; }; } // namespace pipeline } // namespace doris diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.h b/be/src/pipeline/pipeline_x/pipeline_x_task.h index 864709b4ede434b..0342d4b0fcc0e40 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.h @@ -67,14 +67,24 @@ class PipelineXTask : public PipelineTask { Status close() override; bool source_can_read() override { - return _source->can_read(_state) || _ignore_blocking_source(); + if (_pipeline->_always_can_read) { + return true; + } + for (auto& op : _operators) { + if (!op->can_read(_state)) { + return false; + } + } + return true; } bool runtime_filters_are_ready_or_timeout() override { return _source->runtime_filters_are_ready_or_timeout(); } - bool sink_can_write() override { return _sink->can_write(_state) || _ignore_blocking_sink(); } + bool sink_can_write() override { + return _sink->can_write(_state) || _pipeline->_always_can_write; + } Status finalize() override; @@ -105,17 +115,6 @@ class PipelineXTask : public PipelineTask { } private: - [[nodiscard]] bool _ignore_blocking_sink() { return _root->can_terminate_early(_state); } - - [[nodiscard]] bool _ignore_blocking_source() { - for (size_t i = 1; i < _operators.size(); i++) { - if (_operators[i]->can_terminate_early(_state)) { - return true; - } - } - return false; - } - using DependencyMap = std::map; Status _open() override; diff --git a/be/src/pipeline/task_scheduler.cpp b/be/src/pipeline/task_scheduler.cpp index 0be333479b694b4..8b2f8e3f0c76af6 100644 --- a/be/src/pipeline/task_scheduler.cpp +++ b/be/src/pipeline/task_scheduler.cpp @@ -36,6 +36,7 @@ #include "pipeline/task_queue.h" #include "pipeline_fragment_context.h" #include "runtime/query_context.h" +#include "util/debug_util.h" #include "util/sse_util.hpp" #include "util/thread.h" #include "util/threadpool.h" @@ -118,15 +119,14 @@ void BlockedTaskScheduler::_schedule() { _make_task_run(local_blocked_tasks, iter, ready_tasks, PipelineTaskState::PENDING_FINISH); } - } else if (task->fragment_context()->is_canceled()) { + } else if (task->query_context()->is_cancelled()) { _make_task_run(local_blocked_tasks, iter, ready_tasks); } else if (task->query_context()->is_timeout(now)) { LOG(WARNING) << "Timeout, query_id=" << print_id(task->query_context()->query_id()) - << ", instance_id=" - << print_id(task->fragment_context()->get_fragment_instance_id()) + << ", instance_id=" << print_id(task->instance_id()) << ", task info: " << task->debug_string(); - task->fragment_context()->cancel(PPlanFragmentCancelReason::TIMEOUT); + task->query_context()->cancel(true, "", Status::Cancelled("")); _make_task_run(local_blocked_tasks, iter, ready_tasks); } else if (state == PipelineTaskState::BLOCKED_FOR_DEPENDENCY) { if (task->has_dependency()) { @@ -270,7 +270,12 @@ void TaskScheduler::_do_work(size_t index) { task->set_previous_core_id(index); if (!status.ok()) { task->set_eos_time(); - LOG(WARNING) << fmt::format("Pipeline task failed. reason: {}", status.to_string()); + LOG(WARNING) << fmt::format( + "Pipeline task failed. query_id: {} reason: {}", + PrintInstanceStandardInfo(task->query_context()->query_id(), + task->fragment_context()->get_fragment_id(), + task->fragment_context()->get_fragment_instance_id()), + status.to_string()); // Print detail informations below when you debugging here. // // LOG(WARNING)<< "task:\n"<debug_string(); @@ -321,8 +326,8 @@ void TaskScheduler::_try_close_task(PipelineTask* task, PipelineTaskState state) if (!status.ok() && state != PipelineTaskState::CANCELED) { // Call `close` if `try_close` failed to make sure allocated resources are released task->close(); - task->fragment_context()->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, - status.to_string()); + task->query_context()->cancel(true, status.to_string(), + Status::Cancelled(status.to_string())); state = PipelineTaskState::CANCELED; } else if (task->is_pending_finish()) { task->set_state(PipelineTaskState::PENDING_FINISH); @@ -331,8 +336,8 @@ void TaskScheduler::_try_close_task(PipelineTask* task, PipelineTaskState state) } else { status = task->close(); if (!status.ok() && state != PipelineTaskState::CANCELED) { - task->fragment_context()->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, - status.to_string()); + task->query_context()->cancel(true, status.to_string(), + Status::Cancelled(status.to_string())); state = PipelineTaskState::CANCELED; } DCHECK(!task->is_pending_finish()) << task->debug_string(); diff --git a/be/src/runtime/cache/result_node.h b/be/src/runtime/cache/result_node.h index 3edf48261b69e00..31ab19321966660 100644 --- a/be/src/runtime/cache/result_node.h +++ b/be/src/runtime/cache/result_node.h @@ -63,6 +63,9 @@ class PartitionRowBatch { if (req_param.last_version_time() > _cache_value->param().last_version_time()) { return false; } + if (req_param.partition_num() != _cache_value->param().partition_num()) { + return false; + } return true; } @@ -74,9 +77,17 @@ class PartitionRowBatch { if (up_param.last_version_time() > _cache_value->param().last_version_time()) { return true; } + if (up_param.last_version_time() == _cache_value->param().last_version_time() && + up_param.partition_num() != _cache_value->param().partition_num()) { + return true; + } if (up_param.last_version() > _cache_value->param().last_version()) { return true; } + if (up_param.last_version() == _cache_value->param().last_version() && + up_param.partition_num() != _cache_value->param().partition_num()) { + return true; + } return false; } diff --git a/be/src/runtime/exec_env.cpp b/be/src/runtime/exec_env.cpp index 8f63d6fe6a9c44b..eb0337d33176e0b 100644 --- a/be/src/runtime/exec_env.cpp +++ b/be/src/runtime/exec_env.cpp @@ -19,13 +19,120 @@ #include +#include +#include + +#include "common/config.h" +#include "runtime/frontend_info.h" +#include "time.h" +#include "util/debug_util.h" +#include "util/time.h" + namespace doris { -ExecEnv::ExecEnv() : _is_init(false) {} +ExecEnv::ExecEnv() = default; -ExecEnv::~ExecEnv() {} +ExecEnv::~ExecEnv() { + _destroy(); +} const std::string& ExecEnv::token() const { return _master_info->token; } + +std::map ExecEnv::get_frontends() { + std::lock_guard lg(_frontends_lock); + return _frontends; +} + +void ExecEnv::update_frontends(const std::vector& new_fe_infos) { + std::lock_guard lg(_frontends_lock); + + std::set dropped_fes; + + for (const auto& cur_fe : _frontends) { + dropped_fes.insert(cur_fe.first); + } + + for (const auto& coming_fe_info : new_fe_infos) { + auto itr = _frontends.find(coming_fe_info.coordinator_address); + + if (itr == _frontends.end()) { + LOG(INFO) << "A completely new frontend, " << PrintFrontendInfo(coming_fe_info); + + _frontends.insert(std::pair( + coming_fe_info.coordinator_address, + FrontendInfo {coming_fe_info, GetCurrentTimeMicros() / 1000, /*first time*/ + GetCurrentTimeMicros() / 1000 /*last time*/})); + + continue; + } + + dropped_fes.erase(coming_fe_info.coordinator_address); + + if (coming_fe_info.process_uuid == 0) { + LOG(WARNING) << "Frontend " << PrintFrontendInfo(coming_fe_info) + << " is in an unknown state."; + } + + if (coming_fe_info.process_uuid == itr->second.info.process_uuid) { + itr->second.last_reveiving_time_ms = GetCurrentTimeMicros() / 1000; + continue; + } + + // If we get here, means this frontend has already restarted. + itr->second.info.process_uuid = coming_fe_info.process_uuid; + itr->second.first_receiving_time_ms = GetCurrentTimeMicros() / 1000; + itr->second.last_reveiving_time_ms = GetCurrentTimeMicros() / 1000; + LOG(INFO) << "Update frontend " << PrintFrontendInfo(coming_fe_info); + } + + for (const auto& dropped_fe : dropped_fes) { + LOG(INFO) << "Frontend " << PrintThriftNetworkAddress(dropped_fe) + << " has already been dropped, remove it"; + _frontends.erase(dropped_fe); + } +} + +std::map ExecEnv::get_running_frontends() { + std::lock_guard lg(_frontends_lock); + std::map res; + const int expired_duration = config::fe_expire_duration_seconds * 1000; + const auto now = GetCurrentTimeMicros() / 1000; + + for (const auto& pair : _frontends) { + if (pair.second.info.process_uuid != 0) { + if (now - pair.second.last_reveiving_time_ms < expired_duration) { + // If fe info has just been update in last expired_duration, regard it as running. + res[pair.first] = pair.second; + } else { + // Fe info has not been udpate for more than expired_duration, regard it as an abnormal. + // Abnormal means this fe can not connect to master, and it is not dropped from cluster. + // or fe do not have master yet. + LOG(INFO) << "Frontend " << PrintFrontendInfo(pair.second.info) + << " has not update its hb " + << "for more than " << config::fe_expire_duration_seconds + << " secs, regard it as abnormal."; + } + + continue; + } + + if (pair.second.last_reveiving_time_ms - pair.second.first_receiving_time_ms > + expired_duration) { + // A zero process-uuid that sustains more than 60 seconds(default). + // We will regard this fe as a abnormal frontend. + LOG(INFO) << "Frontend " << PrintFrontendInfo(pair.second.info) + << " has not update its hb " + << "for more than " << config::fe_expire_duration_seconds + << " secs, regard it as abnormal."; + continue; + } else { + res[pair.first] = pair.second; + } + } + + return res; +} + } // namespace doris diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 18dd926ba64c544..30c68391666e011 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -17,11 +17,14 @@ #pragma once +#include #include #include +#include #include #include +#include #include #include #include @@ -34,6 +37,7 @@ #include "vec/common/hash_table/phmap_fwd_decl.h" namespace doris { +struct FrontendInfo; namespace vectorized { class VDataStreamMgr; class ScannerScheduler; @@ -76,6 +80,8 @@ class HeartbeatFlags; class FrontendServiceClient; class FileMetaCache; +inline bool k_doris_exit = false; + // Execution environment for queries/plan fragments. // Contains all required global structures, and handles to // singleton services. Clients must call StartServices exactly @@ -94,14 +100,11 @@ class ExecEnv { return &s_exec_env; } - // only used for test - ExecEnv(); - // Empty destructor because the compiler-generated one requires full // declarations for classes in scoped_ptrs. ~ExecEnv(); - bool initialized() const { return _is_init; } + static bool ready() { return _s_ready.load(std::memory_order_acquire); } const std::string& token() const; ExternalScanContextMgr* external_scan_context_mgr() { return _external_scan_context_mgr; } doris::vectorized::VDataStreamMgr* vstream_mgr() { return _vstream_mgr; } @@ -173,9 +176,6 @@ class ExecEnv { const std::vector& store_paths() const { return _store_paths; } - StorageEngine* storage_engine() { return _storage_engine; } - void set_storage_engine(StorageEngine* storage_engine) { _storage_engine = storage_engine; } - std::shared_ptr stream_load_executor() { return _stream_load_executor; } RoutineLoadTaskExecutor* routine_load_task_executor() { return _routine_load_task_executor; } HeartbeatFlags* heartbeat_flags() { return _heartbeat_flags; } @@ -199,7 +199,13 @@ class ExecEnv { this->_stream_load_executor = stream_load_executor; } + void update_frontends(const std::vector& new_infos); + std::map get_frontends(); + std::map get_running_frontends(); + private: + ExecEnv(); + Status _init(const std::vector& store_paths); void _destroy(); @@ -208,7 +214,7 @@ class ExecEnv { void _register_metrics(); void _deregister_metrics(); - bool _is_init; + inline static std::atomic_bool _s_ready {false}; std::vector _store_paths; // Leave protected so that subclasses can override @@ -262,8 +268,6 @@ class ExecEnv { BrpcClientCache* _internal_client_cache = nullptr; BrpcClientCache* _function_client_cache = nullptr; - StorageEngine* _storage_engine = nullptr; - std::shared_ptr _stream_load_executor; RoutineLoadTaskExecutor* _routine_load_task_executor = nullptr; SmallFileMgr* _small_file_mgr = nullptr; @@ -277,6 +281,9 @@ class ExecEnv { std::unique_ptr _global_zone_cache; std::shared_mutex _zone_cache_rw_lock; + + std::mutex _frontends_lock; + std::map _frontends; }; template <> diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index fd6eff6950a8bbb..d3d4df8a343e41e 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -68,11 +68,13 @@ #include "runtime/stream_load/stream_load_executor.h" #include "runtime/task_group/task_group_manager.h" #include "runtime/thread_context.h" +#include "service/backend_options.h" #include "service/point_query_executor.h" #include "util/bfd_parser.h" #include "util/bit_util.h" #include "util/brpc_client_cache.h" #include "util/cpu_info.h" +#include "util/disk_info.h" #include "util/doris_metrics.h" #include "util/mem_info.h" #include "util/metrics.h" @@ -98,15 +100,39 @@ DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(send_batch_thread_pool_queue_size, MetricUnit DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(download_cache_thread_pool_thread_num, MetricUnit::NOUNIT); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(download_cache_thread_pool_queue_size, MetricUnit::NOUNIT); +static void init_doris_metrics(const std::vector& store_paths) { + bool init_system_metrics = config::enable_system_metrics; + std::set disk_devices; + std::vector network_interfaces; + std::vector paths; + for (auto& store_path : store_paths) { + paths.emplace_back(store_path.path); + } + if (init_system_metrics) { + auto st = DiskInfo::get_disk_devices(paths, &disk_devices); + if (!st.ok()) { + LOG(WARNING) << "get disk devices failed, status=" << st; + return; + } + st = get_inet_interfaces(&network_interfaces, BackendOptions::is_bind_ipv6()); + if (!st.ok()) { + LOG(WARNING) << "get inet interfaces failed, status=" << st; + return; + } + } + DorisMetrics::instance()->initialize(init_system_metrics, disk_devices, network_interfaces); +} + Status ExecEnv::init(ExecEnv* env, const std::vector& store_paths) { return env->_init(store_paths); } Status ExecEnv::_init(const std::vector& store_paths) { //Only init once before be destroyed - if (_is_init) { + if (ready()) { return Status::OK(); } + init_doris_metrics(store_paths); _store_paths = store_paths; _external_scan_context_mgr = new ExternalScanContextMgr(this); @@ -190,7 +216,7 @@ Status ExecEnv::_init(const std::vector& store_paths) { RETURN_IF_ERROR(_load_channel_mgr->init(MemInfo::mem_limit())); _heartbeat_flags = new HeartbeatFlags(); _register_metrics(); - _is_init = true; + _s_ready = true; return Status::OK(); } @@ -379,9 +405,11 @@ void ExecEnv::_deregister_metrics() { void ExecEnv::_destroy() { //Only destroy once after init - if (!_is_init) { + if (!ready()) { return; } + // Memory barrier to prevent other threads from accessing destructed resources + _s_ready = false; _deregister_metrics(); SAFE_DELETE(_internal_client_cache); SAFE_DELETE(_function_client_cache); @@ -422,8 +450,6 @@ void ExecEnv::_destroy() { _page_no_cache_mem_tracker.reset(); _brpc_iobuf_block_memory_tracker.reset(); InvertedIndexSearcherCache::reset_global_instance(); - - _is_init = false; } void ExecEnv::destroy(ExecEnv* env) { diff --git a/be/src/runtime/external_scan_context_mgr.cpp b/be/src/runtime/external_scan_context_mgr.cpp index 9bb095d9a6da4ed..2a3dc92521fe67d 100644 --- a/be/src/runtime/external_scan_context_mgr.cpp +++ b/be/src/runtime/external_scan_context_mgr.cpp @@ -103,7 +103,8 @@ Status ExternalScanContextMgr::clear_scan_context(const std::string& context_id) } if (context != nullptr) { // first cancel the fragment instance, just ignore return status - _exec_env->fragment_mgr()->cancel(context->fragment_instance_id); + _exec_env->fragment_mgr()->cancel_instance(context->fragment_instance_id, + PPlanFragmentCancelReason::INTERNAL_ERROR); // clear the fragment instance's related result queue _exec_env->result_queue_mgr()->cancel(context->fragment_instance_id); LOG(INFO) << "close scan context: context id [ " << context_id << " ]"; @@ -143,7 +144,8 @@ void ExternalScanContextMgr::gc_expired_context() { } for (auto expired_context : expired_contexts) { // must cancel the fragment instance, otherwise return thrift transport TTransportException - _exec_env->fragment_mgr()->cancel(expired_context->fragment_instance_id); + _exec_env->fragment_mgr()->cancel_instance(expired_context->fragment_instance_id, + PPlanFragmentCancelReason::INTERNAL_ERROR); _exec_env->result_queue_mgr()->cancel(expired_context->fragment_instance_id); } } diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 75a25a48ff74527..c8943a8a853ac84 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -42,11 +42,13 @@ #include +#include "common/status.h" #include "pipeline/pipeline_x/pipeline_x_fragment_context.h" // IWYU pragma: no_include #include // IWYU pragma: keep #include #include +#include #include #include @@ -61,6 +63,7 @@ #include "runtime/client_cache.h" #include "runtime/descriptors.h" #include "runtime/exec_env.h" +#include "runtime/frontend_info.h" #include "runtime/memory/mem_tracker_limiter.h" #include "runtime/plan_fragment_executor.h" #include "runtime/primitive_type.h" @@ -75,6 +78,7 @@ #include "runtime/thread_context.h" #include "runtime/types.h" #include "service/backend_options.h" +#include "util/debug_util.h" #include "util/doris_metrics.h" #include "util/hash_util.hpp" #include "util/mem_info.h" @@ -503,7 +507,8 @@ Status FragmentMgr::_get_query_ctx(const Params& params, TUniqueId query_id, boo LOG(INFO) << "query_id: " << UniqueId(query_ctx->query_id().hi, query_ctx->query_id().lo) << " coord_addr " << query_ctx->coord_addr - << " total fragment num on current host: " << params.fragment_num_on_host; + << " total fragment num on current host: " << params.fragment_num_on_host + << " fe process uuid: " << params.query_options.fe_process_uuid; query_ctx->query_globals = params.query_globals; if (params.__isset.resource_info) { @@ -842,72 +847,120 @@ void FragmentMgr::_set_scan_concurrency(const Param& params, QueryContext* query #endif } -void FragmentMgr::cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason, - const std::string& msg) { - bool find_the_fragment = false; +void FragmentMgr::cancel_query(const TUniqueId& query_id, const PPlanFragmentCancelReason& reason, + const std::string& msg) { + std::unique_lock state_lock; + return cancel_query_unlocked(query_id, reason, state_lock, msg); +} - std::shared_ptr fragment_executor; - { - std::lock_guard lock(_lock); - auto iter = _fragment_map.find(fragment_id); - if (iter != _fragment_map.end()) { - fragment_executor = iter->second; - } - } - if (fragment_executor) { - find_the_fragment = true; - fragment_executor->cancel(reason, msg); +// Cancel all instances/fragments of query, and set query_ctx of the query canceled at last. +void FragmentMgr::cancel_query_unlocked(const TUniqueId& query_id, + const PPlanFragmentCancelReason& reason, + const std::unique_lock& state_lock, + const std::string& msg) { + auto ctx = _query_ctx_map.find(query_id); + + if (ctx == _query_ctx_map.end()) { + LOG(WARNING) << "Query " << print_id(query_id) << " does not exists, failed to cancel it"; + return; } - std::shared_ptr pipeline_fragment_ctx; - { - std::lock_guard lock(_lock); - auto iter = _pipeline_map.find(fragment_id); - if (iter != _pipeline_map.end()) { - pipeline_fragment_ctx = iter->second; + if (ctx->second->enable_pipeline_exec()) { + for (auto it : ctx->second->fragment_ids) { + // instance_id will not be removed from query_context.instance_ids currently + // and it will be removed from fragment_mgr::_pipeline_map only. + // so we add this check to avoid too many WARNING log. + if (_pipeline_map.contains(it)) { + cancel_instance_unlocked(it, reason, state_lock, msg); + } + } + } else { + for (auto it : ctx->second->fragment_ids) { + cancel_fragment_unlocked(it, reason, state_lock, msg); } - } - if (pipeline_fragment_ctx) { - find_the_fragment = true; - pipeline_fragment_ctx->cancel(reason, msg); } - if (!find_the_fragment) { - LOG(WARNING) << "Do not find the fragment instance id:" << fragment_id << " to cancel"; - } + ctx->second->cancel(true, msg, Status::Cancelled(msg)); + _query_ctx_map.erase(query_id); + LOG(INFO) << "Query " << print_id(query_id) << " is cancelled. Reason: " << msg; } -void FragmentMgr::cancel_query(const TUniqueId& query_id, const PPlanFragmentCancelReason& reason, - const std::string& msg) { - std::vector cancel_fragment_ids; - { - std::lock_guard lock(_lock); - auto ctx = _query_ctx_map.find(query_id); - if (ctx != _query_ctx_map.end()) { - cancel_fragment_ids = ctx->second->fragment_ids; +void FragmentMgr::cancel_fragment(const TUniqueId& fragment_id, + const PPlanFragmentCancelReason& reason, const std::string& msg) { + std::unique_lock state_lock(_lock); + return cancel_fragment_unlocked(fragment_id, reason, state_lock, msg); +} + +void FragmentMgr::cancel_fragment_unlocked(const TUniqueId& fragment_id, + const PPlanFragmentCancelReason& reason, + const std::unique_lock& state_lock, + const std::string& msg) { + return cancel_unlocked_impl(fragment_id, reason, state_lock, false /*not pipeline query*/, msg); +} + +void FragmentMgr::cancel_instance(const TUniqueId& instance_id, + const PPlanFragmentCancelReason& reason, const std::string& msg) { + std::unique_lock state_lock(_lock); + return cancel_instance_unlocked(instance_id, reason, state_lock, msg); +} + +void FragmentMgr::cancel_instance_unlocked(const TUniqueId& instance_id, + const PPlanFragmentCancelReason& reason, + const std::unique_lock& state_lock, + const std::string& msg) { + return cancel_unlocked_impl(instance_id, reason, state_lock, true /*pipeline query*/, msg); +} + +void FragmentMgr::cancel_unlocked_impl(const TUniqueId& id, const PPlanFragmentCancelReason& reason, + const std::unique_lock& /*state_lock*/, + bool is_pipeline, const std::string& msg) { + if (is_pipeline) { + const TUniqueId& instance_id = id; + auto itr = _pipeline_map.find(instance_id); + + if (itr != _pipeline_map.end()) { + // calling PipelineFragmentContext::cancel + itr->second->cancel(reason, msg); + } else { + LOG(WARNING) << "Could not find the instance id:" << print_id(instance_id) + << " to cancel"; + } + } else { + const TUniqueId& fragment_id = id; + auto itr = _fragment_map.find(fragment_id); + + if (itr != _fragment_map.end()) { + // calling PlanFragmentExecutor::cancel + itr->second->cancel(reason, msg); + } else { + LOG(WARNING) << "Could not find the fragment id:" << print_id(fragment_id) + << " to cancel"; } - } - for (auto it : cancel_fragment_ids) { - cancel(it, reason, msg); } } bool FragmentMgr::query_is_canceled(const TUniqueId& query_id) { std::lock_guard lock(_lock); auto ctx = _query_ctx_map.find(query_id); - if (ctx != _query_ctx_map.end()) { - for (auto it : ctx->second->fragment_ids) { - auto fragment_executor_iter = _fragment_map.find(it); - if (fragment_executor_iter != _fragment_map.end() && fragment_executor_iter->second) { - return fragment_executor_iter->second->is_canceled(); - } - auto pipeline_ctx_iter = _pipeline_map.find(it); - if (pipeline_ctx_iter != _pipeline_map.end() && pipeline_ctx_iter->second) { - return pipeline_ctx_iter->second->is_canceled(); + if (ctx != _query_ctx_map.end()) { + const bool is_pipeline_version = ctx->second->enable_pipeline_exec(); + for (auto itr : ctx->second->fragment_ids) { + if (is_pipeline_version) { + auto pipeline_ctx_iter = _pipeline_map.find(itr); + if (pipeline_ctx_iter != _pipeline_map.end() && pipeline_ctx_iter->second) { + return pipeline_ctx_iter->second->is_canceled(); + } + } else { + auto fragment_executor_iter = _fragment_map.find(itr); + if (fragment_executor_iter != _fragment_map.end() && + fragment_executor_iter->second) { + return fragment_executor_iter->second->is_canceled(); + } } } } + return true; } @@ -915,7 +968,7 @@ void FragmentMgr::cancel_worker() { LOG(INFO) << "FragmentMgr cancel worker start working."; do { std::vector to_cancel; - std::vector to_cancel_queries; + std::vector queries_to_cancel; vectorized::VecDateTimeValue now = vectorized::VecDateTimeValue::local_time(); { std::lock_guard lock(_lock); @@ -931,13 +984,45 @@ void FragmentMgr::cancel_worker() { ++it; } } + + const auto& running_fes = ExecEnv::GetInstance()->get_running_frontends(); + for (const auto& q : _query_ctx_map) { + auto itr = running_fes.find(q.second->coord_addr); + if (itr != running_fes.end()) { + // We use conservative strategy. + // 1. If same process uuid, do not cancel + // 2. If fe has zero process uuid, do not cancel + // 3. If query's process uuid is zero, do not cancel + if (q.second->get_fe_process_uuid() == itr->second.info.process_uuid || + itr->second.info.process_uuid == 0 || + q.second->get_fe_process_uuid() == 0) { + continue; + } + } + + // Coorninator of this query has already dead. + queries_to_cancel.push_back(q.first); + } } + + // TODO(zhiqiang): It seems that timeout_canceled_fragment_count is + // designed to count canceled fragment of non-pipeline query. timeout_canceled_fragment_count->increment(to_cancel.size()); for (auto& id : to_cancel) { - cancel(id, PPlanFragmentCancelReason::TIMEOUT); + cancel_fragment(id, PPlanFragmentCancelReason::TIMEOUT); LOG(INFO) << "FragmentMgr cancel worker going to cancel timeout fragment " << print_id(id); } + + if (!queries_to_cancel.empty()) { + LOG(INFO) << "There are " << queries_to_cancel.size() + << " queries need to be cancelled, coordinator dead."; + } + + for (const auto& qid : queries_to_cancel) { + cancel_query(qid, PPlanFragmentCancelReason::INTERNAL_ERROR, + std::string("Coordinator dead.")); + } } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(1))); LOG(INFO) << "FragmentMgr cancel worker is going to exit."; } diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h index cdc01627a0a279a..8548d19d7875731 100644 --- a/be/src/runtime/fragment_mgr.h +++ b/be/src/runtime/fragment_mgr.h @@ -93,15 +93,28 @@ class FragmentMgr : public RestMonitorIface { Status start_query_execution(const PExecPlanFragmentStartRequest* request); - void cancel(const TUniqueId& fragment_id) { - cancel(fragment_id, PPlanFragmentCancelReason::INTERNAL_ERROR); - } - - void cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason, - const std::string& msg = ""); - + // This method can only be used to cancel a fragment of non-pipeline query. + void cancel_fragment(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason, + const std::string& msg = ""); + void cancel_fragment_unlocked(const TUniqueId& instance_id, + const PPlanFragmentCancelReason& reason, + const std::unique_lock& state_lock, + const std::string& msg = ""); + + // Pipeline version, cancel a fragment instance. + void cancel_instance(const TUniqueId& instance_id, const PPlanFragmentCancelReason& reason, + const std::string& msg = ""); + void cancel_instance_unlocked(const TUniqueId& instance_id, + const PPlanFragmentCancelReason& reason, + const std::unique_lock& state_lock, + const std::string& msg = ""); + + // Can be used in both version. void cancel_query(const TUniqueId& query_id, const PPlanFragmentCancelReason& reason, const std::string& msg = ""); + void cancel_query_unlocked(const TUniqueId& query_id, const PPlanFragmentCancelReason& reason, + const std::unique_lock& state_lock, + const std::string& msg = ""); bool query_is_canceled(const TUniqueId& query_id); @@ -132,6 +145,10 @@ class FragmentMgr : public RestMonitorIface { ThreadPool* get_thread_pool() { return _thread_pool.get(); } private: + void cancel_unlocked_impl(const TUniqueId& id, const PPlanFragmentCancelReason& reason, + const std::unique_lock& state_lock, bool is_pipeline, + const std::string& msg = ""); + void _exec_actual(std::shared_ptr fragment_executor, const FinishCallback& cb); diff --git a/be/src/runtime/result_writer.cpp b/be/src/runtime/frontend_info.h similarity index 79% rename from be/src/runtime/result_writer.cpp rename to be/src/runtime/frontend_info.h index b5537e486cb7e04..c16d63096fb5c5b 100644 --- a/be/src/runtime/result_writer.cpp +++ b/be/src/runtime/frontend_info.h @@ -15,12 +15,16 @@ // specific language governing permissions and limitations // under the License. -#include "runtime/result_writer.h" +#include -namespace doris { +#include -const std::string ResultWriter::NULL_IN_CSV = "\\N"; +namespace doris { -} +struct FrontendInfo { + TFrontendInfo info; + std::time_t first_receiving_time_ms; + std::time_t last_reveiving_time_ms; +}; -/* vim: set ts=4 sw=4 sts=4 tw=100 expandtab : */ +} // namespace doris diff --git a/be/src/runtime/load_stream.cpp b/be/src/runtime/load_stream.cpp index 961f73540a4e94a..5abf75476cbf5dd 100644 --- a/be/src/runtime/load_stream.cpp +++ b/be/src/runtime/load_stream.cpp @@ -430,6 +430,7 @@ int LoadStream::on_received_messages(StreamId id, butil::IOBuf* const messages[] auto st = close(hdr.src_id(), tablets_to_commit, &success_tablet_ids, &failed_tablet_ids); _report_result(id, st, &success_tablet_ids, &failed_tablet_ids); + brpc::StreamClose(id); } break; default: LOG(WARNING) << "unexpected stream message " << hdr.opcode(); diff --git a/be/src/runtime/memory/cache_manager.cpp b/be/src/runtime/memory/cache_manager.cpp index 027ed81b163d2e3..eada36836d5518d 100644 --- a/be/src/runtime/memory/cache_manager.cpp +++ b/be/src/runtime/memory/cache_manager.cpp @@ -43,7 +43,16 @@ int64_t CacheManager::for_each_cache_prune_stale(RuntimeProfile* profile) { int64_t CacheManager::for_each_cache_prune_all(RuntimeProfile* profile) { return for_each_cache_prune_stale_wrap( - [](CachePolicy* cache_policy) { cache_policy->prune_all(); }, profile); + [](CachePolicy* cache_policy) { cache_policy->prune_all(false); }, profile); +} + +void CacheManager::clear_once(CachePolicy::CacheType type) { + std::lock_guard l(_caches_lock); + for (auto cache_policy : _caches) { + if (cache_policy->type() == type) { + cache_policy->prune_all(true); // will print log + } + } } } // namespace doris diff --git a/be/src/runtime/memory/cache_manager.h b/be/src/runtime/memory/cache_manager.h index 6086c02b94fba39..fd7d5875b0d1cf3 100644 --- a/be/src/runtime/memory/cache_manager.h +++ b/be/src/runtime/memory/cache_manager.h @@ -17,12 +17,11 @@ #pragma once +#include "runtime/memory/cache_policy.h" #include "util/runtime_profile.h" namespace doris { -class CachePolicy; - // Hold the list of all caches, for prune when memory not enough or timing. class CacheManager { public: @@ -53,6 +52,8 @@ class CacheManager { int64_t for_each_cache_prune_all(RuntimeProfile* profile = nullptr); + void clear_once(CachePolicy::CacheType type); + private: static inline CacheManager* _s_instance = nullptr; diff --git a/be/src/runtime/memory/cache_policy.cpp b/be/src/runtime/memory/cache_policy.cpp new file mode 100644 index 000000000000000..e79beaffa806ad4 --- /dev/null +++ b/be/src/runtime/memory/cache_policy.cpp @@ -0,0 +1,34 @@ +// 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 "runtime/memory/cache_policy.h" + +#include "runtime/memory/cache_manager.h" + +namespace doris { + +CachePolicy::CachePolicy(CacheType type, uint32_t stale_sweep_time_s) + : _type(type), _stale_sweep_time_s(stale_sweep_time_s) { + _it = CacheManager::instance()->register_cache(this); + init_profile(); +} + +CachePolicy::~CachePolicy() { + CacheManager::instance()->unregister_cache(_it); +} + +} // namespace doris diff --git a/be/src/runtime/memory/cache_policy.h b/be/src/runtime/memory/cache_policy.h index 14308088e619336..008e5f4a47fbf19 100644 --- a/be/src/runtime/memory/cache_policy.h +++ b/be/src/runtime/memory/cache_policy.h @@ -17,7 +17,6 @@ #pragma once -#include "runtime/memory/cache_manager.h" #include "util/runtime_profile.h" namespace doris { @@ -27,21 +26,54 @@ static constexpr int32_t CACHE_MIN_FREE_SIZE = 67108864; // 64M // Base of all caches. register to CacheManager when cache is constructed. class CachePolicy { public: - CachePolicy(const std::string& name, uint32_t stale_sweep_time_s) - : _name(name), _stale_sweep_time_s(stale_sweep_time_s) { - _it = CacheManager::instance()->register_cache(this); - init_profile(); + enum class CacheType { + DATA_PAGE_CACHE = 0, + INDEXPAGE_CACHE = 1, + PK_INDEX_PAGE_CACHE = 2, + SCHEMA_CACHE = 3, + SEGMENT_CACHE = 4, + INVERTEDINDEX_SEARCHER_CACHE = 5, + INVERTEDINDEX_QUERY_CACHE = 6, + LOOKUP_CONNECTION_CACHE = 7 + }; + + static std::string type_string(CacheType type) { + switch (type) { + case CacheType::DATA_PAGE_CACHE: + return "DataPageCache"; + case CacheType::INDEXPAGE_CACHE: + return "IndexPageCache"; + case CacheType::PK_INDEX_PAGE_CACHE: + return "PKIndexPageCache"; + case CacheType::SCHEMA_CACHE: + return "SchemaCache"; + case CacheType::SEGMENT_CACHE: + return "SegmentCache"; + case CacheType::INVERTEDINDEX_SEARCHER_CACHE: + return "InvertedIndexSearcherCache"; + case CacheType::INVERTEDINDEX_QUERY_CACHE: + return "InvertedIndexQueryCache"; + case CacheType::LOOKUP_CONNECTION_CACHE: + return "LookupConnectionCache"; + default: + LOG(FATAL) << "not match type of cache policy :" << static_cast(type); + } + __builtin_unreachable(); } - virtual ~CachePolicy() { CacheManager::instance()->unregister_cache(_it); }; + CachePolicy(CacheType type, uint32_t stale_sweep_time_s); + virtual ~CachePolicy(); + virtual void prune_stale() = 0; - virtual void prune_all() = 0; + virtual void prune_all(bool clear) = 0; + CacheType type() { return _type; } RuntimeProfile* profile() { return _profile.get(); } protected: void init_profile() { - _profile = std::make_unique(fmt::format("Cache name={}", _name)); + _profile = + std::make_unique(fmt::format("Cache type={}", type_string(_type))); _prune_stale_number_counter = ADD_COUNTER(_profile, "PruneStaleNumber", TUnit::UNIT); _prune_all_number_counter = ADD_COUNTER(_profile, "PruneAllNumber", TUnit::UNIT); _freed_memory_counter = ADD_COUNTER(_profile, "FreedMemory", TUnit::BYTES); @@ -49,7 +81,7 @@ class CachePolicy { _cost_timer = ADD_TIMER(_profile, "CostTime"); } - std::string _name; + CacheType _type; std::list::iterator _it; std::unique_ptr _profile; diff --git a/be/src/runtime/memory/lru_cache_policy.h b/be/src/runtime/memory/lru_cache_policy.h index fd900bea6c79eeb..e7b9680eb60f340 100644 --- a/be/src/runtime/memory/lru_cache_policy.h +++ b/be/src/runtime/memory/lru_cache_policy.h @@ -34,14 +34,16 @@ struct LRUCacheValueBase { // Base of lru cache, allow prune stale entry and prune all entry. class LRUCachePolicy : public CachePolicy { public: - LRUCachePolicy(const std::string& name, uint32_t stale_sweep_time_s) - : CachePolicy(name, stale_sweep_time_s) {}; - LRUCachePolicy(const std::string& name, size_t capacity, LRUCacheType type, + LRUCachePolicy(CacheType type, uint32_t stale_sweep_time_s) + : CachePolicy(type, stale_sweep_time_s) {}; + LRUCachePolicy(CacheType type, size_t capacity, LRUCacheType lru_cache_type, uint32_t stale_sweep_time_s, uint32_t num_shards = -1) - : CachePolicy(name, stale_sweep_time_s) { + : CachePolicy(type, stale_sweep_time_s) { _cache = num_shards == -1 - ? std::unique_ptr(new_lru_cache(name, capacity, type)) - : std::unique_ptr(new_lru_cache(name, capacity, type, num_shards)); + ? std::unique_ptr( + new_lru_cache(type_string(type), capacity, lru_cache_type)) + : std::unique_ptr(new_lru_cache(type_string(type), capacity, + lru_cache_type, num_shards)); } ~LRUCachePolicy() override = default; @@ -66,23 +68,26 @@ class LRUCachePolicy : public CachePolicy { COUNTER_SET(_freed_entrys_counter, _cache->prune_if(pred, true)); COUNTER_SET(_freed_memory_counter, byte_size); COUNTER_UPDATE(_prune_stale_number_counter, 1); - LOG(INFO) << fmt::format("{} prune stale {} entries, {} bytes, {} times prune", _name, - _freed_entrys_counter->value(), _freed_memory_counter->value(), + LOG(INFO) << fmt::format("{} prune stale {} entries, {} bytes, {} times prune", + type_string(_type), _freed_entrys_counter->value(), + _freed_memory_counter->value(), _prune_stale_number_counter->value()); } } - void prune_all() override { - if (_cache->mem_consumption() > CACHE_MIN_FREE_SIZE) { + void prune_all(bool clear) override { + if ((clear && _cache->mem_consumption() != 0) || + _cache->mem_consumption() > CACHE_MIN_FREE_SIZE) { COUNTER_SET(_cost_timer, (int64_t)0); SCOPED_TIMER(_cost_timer); auto size = _cache->mem_consumption(); COUNTER_SET(_freed_entrys_counter, _cache->prune()); COUNTER_SET(_freed_memory_counter, size); COUNTER_UPDATE(_prune_all_number_counter, 1); - LOG(INFO) << fmt::format("{} prune all {} entries, {} bytes, {} times prune", _name, - _freed_entrys_counter->value(), _freed_memory_counter->value(), - _prune_stale_number_counter->value()); + LOG(INFO) << fmt::format( + "{} prune all {} entries, {} bytes, {} times prune, is clear: {}", + type_string(_type), _freed_entrys_counter->value(), + _freed_memory_counter->value(), _prune_stale_number_counter->value(), clear); } } diff --git a/be/src/runtime/memory/mem_tracker_limiter.cpp b/be/src/runtime/memory/mem_tracker_limiter.cpp index fe2639ab5e0f2ba..05dfde7dc0ce3ad 100644 --- a/be/src/runtime/memory/mem_tracker_limiter.cpp +++ b/be/src/runtime/memory/mem_tracker_limiter.cpp @@ -87,7 +87,7 @@ MemTrackerLimiter::~MemTrackerLimiter() { // nor can it guarantee that the memory alloc and free are recorded in a one-to-one correspondence. // In order to ensure `consumption of all limiter trackers` + `orphan tracker consumption` = `process tracker consumption` // in real time. Merge its consumption into orphan when parent is process, to avoid repetition. - if (ExecEnv::GetInstance()->initialized()) { + if (ExecEnv::ready()) { ExecEnv::GetInstance()->orphan_mem_tracker()->consume(_consumption->current_value()); } _consumption->set(0); diff --git a/be/src/runtime/memory/thread_mem_tracker_mgr.cpp b/be/src/runtime/memory/thread_mem_tracker_mgr.cpp index d53a3731d836d8c..92974c73a250335 100644 --- a/be/src/runtime/memory/thread_mem_tracker_mgr.cpp +++ b/be/src/runtime/memory/thread_mem_tracker_mgr.cpp @@ -47,7 +47,7 @@ void ThreadMemTrackerMgr::detach_limiter_tracker( } void ThreadMemTrackerMgr::cancel_fragment(const std::string& exceed_msg) { - ExecEnv::GetInstance()->fragment_mgr()->cancel( + ExecEnv::GetInstance()->fragment_mgr()->cancel_instance( _fragment_instance_id, PPlanFragmentCancelReason::MEMORY_LIMIT_EXCEED, exceed_msg); } diff --git a/be/src/runtime/memory/thread_mem_tracker_mgr.h b/be/src/runtime/memory/thread_mem_tracker_mgr.h index 14da64c810056f9..cb5193c2f7e0daf 100644 --- a/be/src/runtime/memory/thread_mem_tracker_mgr.h +++ b/be/src/runtime/memory/thread_mem_tracker_mgr.h @@ -36,8 +36,6 @@ namespace doris { -extern bool k_doris_run; - // Memory Hook is counted in the memory tracker of the current thread. class ThreadMemTrackerMgr { public: @@ -109,7 +107,7 @@ class ThreadMemTrackerMgr { } private: - // is false: ExecEnv::GetInstance()->initialized() = false when thread local is initialized + // is false: ExecEnv::ready() = false when thread local is initialized bool _init = false; // Cache untracked mem. int64_t _untracked_mem = 0; @@ -164,7 +162,7 @@ inline void ThreadMemTrackerMgr::pop_consumer_tracker() { inline void ThreadMemTrackerMgr::consume(int64_t size, bool large_memory_check) { _untracked_mem += size; - if (!k_doris_run || !ExecEnv::GetInstance()->initialized()) { + if (!ExecEnv::ready()) { return; } // When some threads `0 < _untracked_mem < config::mem_tracker_consume_min_size_bytes` diff --git a/be/src/runtime/primitive_type.h b/be/src/runtime/primitive_type.h index 5a4c05ef400449e..07ef91f11cbca48 100644 --- a/be/src/runtime/primitive_type.h +++ b/be/src/runtime/primitive_type.h @@ -91,6 +91,10 @@ constexpr bool is_string_type(PrimitiveType type) { return type == TYPE_CHAR || type == TYPE_VARCHAR || type == TYPE_STRING; } +constexpr bool is_variant_string_type(PrimitiveType type) { + return type == TYPE_VARCHAR || type == TYPE_STRING; +} + constexpr bool is_float_or_double(PrimitiveType type) { return type == TYPE_FLOAT || type == TYPE_DOUBLE; } @@ -110,7 +114,7 @@ TTypeDesc gen_type_desc(const TPrimitiveType::type val); TTypeDesc gen_type_desc(const TPrimitiveType::type val, const std::string& name); template -constexpr PrimitiveType PredicateEvaluateType = is_string_type(type) ? TYPE_STRING : type; +constexpr PrimitiveType PredicateEvaluateType = is_variant_string_type(type) ? TYPE_STRING : type; template struct PrimitiveTypeTraits; diff --git a/be/src/runtime/query_context.h b/be/src/runtime/query_context.h index dbb8c9b1c0319e6..7882b21c8d76f7b 100644 --- a/be/src/runtime/query_context.h +++ b/be/src/runtime/query_context.h @@ -133,11 +133,31 @@ class QueryContext { } [[nodiscard]] bool is_cancelled() const { return _is_cancelled.load(); } - void set_is_cancelled(bool v) { + bool cancel(bool v, std::string msg, Status new_status) { + if (_is_cancelled) { + return false; + } _is_cancelled.store(v); - // Create a error status, so that we could print error stack, and - // we could know which path call cancel. - LOG(INFO) << "task is cancelled, st = " << Status::Error(""); + + set_ready_to_execute(true); + set_exec_status(new_status); + return true; + } + + void set_exec_status(Status new_status) { + if (new_status.ok()) { + return; + } + std::lock_guard l(_exec_status_lock); + if (!_exec_status.ok()) { + return; + } + _exec_status = new_status; + } + + [[nodiscard]] Status exec_status() { + std::lock_guard l(_exec_status_lock); + return _exec_status; } void set_ready_to_execute_only() { @@ -197,6 +217,8 @@ class QueryContext { return _query_options.be_exec_version; } + [[nodiscard]] int64_t get_fe_process_uuid() const { return _query_options.fe_process_uuid; } + RuntimeFilterMgr* runtime_filter_mgr() { return _runtime_filter_mgr.get(); } TUniqueId query_id() const { return _query_id; } @@ -254,6 +276,11 @@ class QueryContext { taskgroup::TaskGroupPtr _task_group; std::unique_ptr _runtime_filter_mgr; const TQueryOptions _query_options; + + std::mutex _exec_status_lock; + // All pipeline tasks use the same query context to report status. So we need a `_exec_status` + // to report the real message if failed. + Status _exec_status = Status::OK(); }; } // namespace doris diff --git a/be/src/runtime/record_batch_queue.h b/be/src/runtime/record_batch_queue.h index 7ababc9b611d307..7528b85f09d69be 100644 --- a/be/src/runtime/record_batch_queue.h +++ b/be/src/runtime/record_batch_queue.h @@ -59,6 +59,8 @@ class RecordBatchQueue { // Shut down the queue. Wakes up all threads waiting on blocking_get or blocking_put. void shutdown(); + size_t size() { return _queue.get_size(); } + private: BlockingQueue> _queue; SpinLock _status_lock; diff --git a/be/src/runtime/result_writer.h b/be/src/runtime/result_writer.h index a1458f1a71b507d..b6cdd10c3a4d906 100644 --- a/be/src/runtime/result_writer.h +++ b/be/src/runtime/result_writer.h @@ -34,7 +34,6 @@ class RuntimeState; class ResultWriter { public: ResultWriter() = default; - ResultWriter(bool output_object_data) : _output_object_data(output_object_data) {} virtual ~ResultWriter() = default; virtual Status init(RuntimeState* state) = 0; @@ -43,7 +42,7 @@ class ResultWriter { virtual int64_t get_written_rows() const { return _written_rows; } - virtual bool output_object_data() const { return _output_object_data; } + bool output_object_data() const { return _output_object_data; } virtual Status append_block(vectorized::Block& block) = 0; @@ -53,17 +52,9 @@ class ResultWriter { _output_object_data = output_object_data; } - static const std::string NULL_IN_CSV; - virtual void set_header_info(const std::string& header_type, const std::string& header) { - _header_type = header_type; - _header = header; - } - protected: int64_t _written_rows = 0; // number of rows written bool _output_object_data = false; - std::string _header_type; - std::string _header; }; } // namespace doris diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index 3efce72ea1bda70..13ad078b5ef472b 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -31,6 +31,7 @@ #include "common/object_pool.h" #include "common/status.h" #include "pipeline/exec/operator.h" +#include "pipeline/pipeline_x/operator.h" #include "runtime/exec_env.h" #include "runtime/load_path_mgr.h" #include "runtime/memory/mem_tracker_limiter.h" @@ -305,7 +306,7 @@ void RuntimeState::get_unreported_errors(std::vector* new_errors) { } bool RuntimeState::is_cancelled() const { - return _is_cancelled.load(); + return _is_cancelled.load() || (_query_ctx && _query_ctx->is_cancelled()); } Status RuntimeState::set_mem_limit_exceeded(const std::string& msg) { @@ -416,24 +417,24 @@ int64_t RuntimeState::get_load_mem_limit() { } void RuntimeState::emplace_local_state( - int id, std::shared_ptr state) { + int id, std::shared_ptr state) { std::unique_lock l(_local_state_lock); _op_id_to_local_state.emplace(id, state); } -std::shared_ptr RuntimeState::get_local_state(int id) { +std::shared_ptr RuntimeState::get_local_state(int id) { std::unique_lock l(_local_state_lock); DCHECK(_op_id_to_local_state.find(id) != _op_id_to_local_state.end()); return _op_id_to_local_state[id]; } void RuntimeState::emplace_sink_local_state( - int id, std::shared_ptr state) { + int id, std::shared_ptr state) { std::unique_lock l(_local_sink_state_lock); _op_id_to_sink_local_state.emplace(id, state); } -std::shared_ptr RuntimeState::get_sink_local_state( +std::shared_ptr RuntimeState::get_sink_local_state( int id) { std::unique_lock l(_local_sink_state_lock); DCHECK(_op_id_to_sink_local_state.find(id) != _op_id_to_sink_local_state.end()); diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index add158dafa7cc6e..76e22fe084b0c6d 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -40,14 +40,15 @@ #include "common/factory_creator.h" #include "common/status.h" #include "gutil/integral_types.h" +#include "util/debug_util.h" #include "util/runtime_profile.h" #include "util/telemetry/telemetry.h" namespace doris { namespace pipeline { -class PipelineXLocalState; -class PipelineXSinkLocalState; +class PipelineXLocalStateBase; +class PipelineXSinkLocalStateBase; } // namespace pipeline class DescriptorTbl; @@ -173,7 +174,9 @@ class RuntimeState { _is_cancelled.store(v); // Create a error status, so that we could print error stack, and // we could know which path call cancel. - LOG(INFO) << "task is cancelled, st = " << Status::Error(msg); + LOG(WARNING) << "Task is cancelled, instance: " + << PrintInstanceStandardInfo(_query_id, _fragment_id, _fragment_instance_id) + << " st = " << Status::Error(msg); } void set_backend_id(int64_t backend_id) { _backend_id = backend_id; } @@ -434,14 +437,20 @@ class RuntimeState { : 0; } - void emplace_local_state(int id, std::shared_ptr state); + inline bool enable_delete_sub_pred_v2() const { + return _query_options.__isset.enable_delete_sub_predicate_v2 && + _query_options.enable_delete_sub_predicate_v2; + } + + void emplace_local_state(int id, + std::shared_ptr state); - std::shared_ptr get_local_state(int id); + std::shared_ptr get_local_state(int id); - void emplace_sink_local_state(int id, - std::shared_ptr state); + void emplace_sink_local_state( + int id, std::shared_ptr state); - std::shared_ptr get_sink_local_state(int id); + std::shared_ptr get_sink_local_state(int id); private: Status create_error_log_file(); @@ -539,8 +548,8 @@ class RuntimeState { std::vector _tablet_commit_infos; std::vector _error_tablet_infos; - std::map> _op_id_to_local_state; - std::map> + std::map> _op_id_to_local_state; + std::map> _op_id_to_sink_local_state; std::mutex _local_state_lock; diff --git a/be/src/runtime/snapshot_loader.cpp b/be/src/runtime/snapshot_loader.cpp index bfcd33233fa5e3e..aff4dd7f2ae57fd 100644 --- a/be/src/runtime/snapshot_loader.cpp +++ b/be/src/runtime/snapshot_loader.cpp @@ -251,7 +251,7 @@ Status SnapshotLoader::download(const std::map& src_to } TabletSharedPtr tablet = - _env->storage_engine()->tablet_manager()->get_tablet(local_tablet_id); + StorageEngine::instance()->tablet_manager()->get_tablet(local_tablet_id); if (tablet == nullptr) { std::stringstream ss; ss << "failed to get local tablet: " << local_tablet_id; @@ -525,7 +525,7 @@ Status SnapshotLoader::remote_http_download( auto local_tablet_id = remote_tablet_snapshot.local_tablet_id; TabletSharedPtr tablet = - _env->storage_engine()->tablet_manager()->get_tablet(local_tablet_id); + StorageEngine::instance()->tablet_manager()->get_tablet(local_tablet_id); if (tablet == nullptr) { std::stringstream ss; ss << "failed to get local tablet: " << local_tablet_id; diff --git a/be/src/runtime/stream_load/stream_load_context.h b/be/src/runtime/stream_load/stream_load_context.h index 7e460d4f6fc7f80..e5eda7496479070 100644 --- a/be/src/runtime/stream_load/stream_load_context.h +++ b/be/src/runtime/stream_load/stream_load_context.h @@ -37,6 +37,7 @@ #include "common/utils.h" #include "runtime/exec_env.h" #include "runtime/stream_load/stream_load_executor.h" +#include "util/byte_buffer.h" #include "util/time.h" #include "util/uid_util.h" @@ -159,9 +160,9 @@ class StreamLoadContext { int64_t txn_id = -1; - // TODO delete code - // for local file - // std::string path; + // http stream + bool need_schema = false; + bool is_read_schema = true; std::string txn_operation = ""; @@ -174,6 +175,9 @@ class StreamLoadContext { std::shared_ptr body_sink; std::shared_ptr pipe; + + ByteBufferPtr schema_buffer = ByteBuffer::allocate(config::stream_tvf_buffer_size); + TStreamLoadPutResult put_result; TStreamLoadMultiTablePutResult multi_table_put_result; diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp index 300a7a40e9eb3af..2e21ec92c349c3b 100644 --- a/be/src/runtime/tablets_channel.cpp +++ b/be/src/runtime/tablets_channel.cpp @@ -436,9 +436,11 @@ Status TabletsChannel::add_batch(const PTabletWriterAddBlockRequest& request, } } - auto get_send_data = [&]() { return vectorized::Block(request.block()); }; - - auto send_data = get_send_data(); + vectorized::Block send_data; + RETURN_IF_ERROR(send_data.deserialize(request.block())); + CHECK(send_data.rows() == request.tablet_ids_size()) + << "block rows: " << send_data.rows() + << ", tablet_ids_size: " << request.tablet_ids_size(); auto write_tablet_data = [&](uint32_t tablet_id, std::function write_func) { diff --git a/be/src/runtime/thread_context.h b/be/src/runtime/thread_context.h index e123ac2aabc668e..a5c9473d665a0c6 100644 --- a/be/src/runtime/thread_context.h +++ b/be/src/runtime/thread_context.h @@ -99,8 +99,6 @@ class ThreadContext; class MemTracker; class RuntimeState; -inline bool k_doris_run = false; -extern bool k_doris_exit; extern bthread_key_t btls_key; // Using gcc11 compiles thread_local variable on lower versions of GLIBC will report an error, @@ -156,8 +154,7 @@ class ThreadContext { public: ThreadContext() { thread_mem_tracker_mgr.reset(new ThreadMemTrackerMgr()); - if (doris::k_doris_run && ExecEnv::GetInstance()->initialized()) - thread_mem_tracker_mgr->init(); + if (ExecEnv::ready()) thread_mem_tracker_mgr->init(); } ~ThreadContext() { thread_context_ptr.init = false; } @@ -238,7 +235,7 @@ class SwitchBthreadLocal { // The brpc server should respond as quickly as possible. bthread_context->thread_mem_tracker_mgr->disable_wait_gc(); // set the data so that next time bthread_getspecific in the thread returns the data. - CHECK((0 == bthread_setspecific(btls_key, bthread_context)) || doris::k_doris_exit); + CHECK(0 == bthread_setspecific(btls_key, bthread_context) || k_doris_exit); thread_context_ptr.init = true; } bthread_id = bthread_self(); @@ -391,7 +388,7 @@ class AddThreadMemTrackerConsumer { do { \ if (doris::thread_context_ptr.init) { \ doris::thread_context()->consume_memory(size); \ - } else if (doris::k_doris_run && doris::ExecEnv::GetInstance()->initialized()) { \ + } else if (doris::ExecEnv::ready()) { \ doris::ExecEnv::GetInstance()->orphan_mem_tracker_raw()->consume_no_update_peak(size); \ } \ } while (0) @@ -399,7 +396,7 @@ class AddThreadMemTrackerConsumer { do { \ if (doris::thread_context_ptr.init) { \ doris::thread_context()->consume_memory(-size); \ - } else if (doris::k_doris_run && doris::ExecEnv::GetInstance()->initialized()) { \ + } else if (doris::ExecEnv::ready()) { \ doris::ExecEnv::GetInstance()->orphan_mem_tracker_raw()->consume_no_update_peak( \ -size); \ } \ diff --git a/be/src/runtime/types.cpp b/be/src/runtime/types.cpp index a919995369ebd99..1b8870286e42c24 100644 --- a/be/src/runtime/types.cpp +++ b/be/src/runtime/types.cpp @@ -40,7 +40,7 @@ TypeDescriptor::TypeDescriptor(const std::vector& types, int* idx) switch (node.type) { case TTypeNodeType::SCALAR: { DCHECK(node.__isset.scalar_type); - const TScalarType scalar_type = node.scalar_type; + const TScalarType& scalar_type = node.scalar_type; type = thrift_to_type(scalar_type.type); if (type == TYPE_CHAR || type == TYPE_VARCHAR || type == TYPE_HLL) { DCHECK(scalar_type.__isset.len); diff --git a/be/src/runtime/user_function_cache.cpp b/be/src/runtime/user_function_cache.cpp index 9edc4dbd46b65c2..39507ec222d5bb0 100644 --- a/be/src/runtime/user_function_cache.cpp +++ b/be/src/runtime/user_function_cache.cpp @@ -151,7 +151,7 @@ Status UserFunctionCache::_load_entry_from_lib(const std::string& dir, const std file); } - std::vector split_parts = strings::Split(file, "."); + std::vector split_parts = _split_string_by_checksum(file); if (split_parts.size() != 3 && split_parts.size() != 4) { return Status::InternalError( "user function's name should be function_id.checksum[.file_name].file_type, now " @@ -379,4 +379,27 @@ Status UserFunctionCache::get_jarpath(int64_t fid, const std::string& url, return Status::OK(); } +std::vector UserFunctionCache::_split_string_by_checksum(const std::string& file) { + std::vector result; + + // Find the first dot from the start + size_t firstDot = file.find('.'); + if (firstDot == std::string::npos) return {}; + + // Find the second dot starting from the first dot's position + size_t secondDot = file.find('.', firstDot + 1); + if (secondDot == std::string::npos) return {}; + + // Find the last dot from the end + size_t lastDot = file.rfind('.'); + if (lastDot == std::string::npos || lastDot <= secondDot) return {}; + + // Split based on these dots + result.push_back(file.substr(0, firstDot)); + result.push_back(file.substr(firstDot + 1, secondDot - firstDot - 1)); + result.push_back(file.substr(secondDot + 1, lastDot - secondDot - 1)); + result.push_back(file.substr(lastDot + 1)); + + return result; +} } // namespace doris diff --git a/be/src/runtime/user_function_cache.h b/be/src/runtime/user_function_cache.h index e58f02294b438ca..5d1bff8b8664b75 100644 --- a/be/src/runtime/user_function_cache.h +++ b/be/src/runtime/user_function_cache.h @@ -74,6 +74,7 @@ class UserFunctionCache { std::string _get_real_url(const std::string& url); std::string _get_file_name_from_url(const std::string& url) const; + std::vector _split_string_by_checksum(const std::string& file); private: std::string _lib_dir; diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index 5b4ca8f07514a61..61c9e80dcd448cc 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -85,7 +85,8 @@ using apache::thrift::concurrency::ThreadFactory; BackendService::BackendService(ExecEnv* exec_env) : _exec_env(exec_env), _agent_server(new AgentServer(exec_env, *exec_env->master_info())) {} -Status BackendService::create_service(ExecEnv* exec_env, int port, ThriftServer** server) { +Status BackendService::create_service(ExecEnv* exec_env, int port, + std::unique_ptr* server) { std::shared_ptr handler(new BackendService(exec_env)); // TODO: do we want a BoostThreadFactory? // TODO: we want separate thread factories here, so that fe requests can't starve @@ -94,7 +95,8 @@ Status BackendService::create_service(ExecEnv* exec_env, int port, ThriftServer* std::shared_ptr be_processor(new BackendServiceProcessor(handler)); - *server = new ThriftServer("backend", be_processor, port, config::be_service_threads); + *server = std::make_unique("backend", be_processor, port, + config::be_service_threads); LOG(INFO) << "Doris BackendService listening on " << port; @@ -118,7 +120,8 @@ Status BackendService::start_plan_fragment_execution(const TExecPlanFragmentPara void BackendService::cancel_plan_fragment(TCancelPlanFragmentResult& return_val, const TCancelPlanFragmentParams& params) { LOG(INFO) << "cancel_plan_fragment(): instance_id=" << params.fragment_instance_id; - _exec_env->fragment_mgr()->cancel(params.fragment_instance_id); + _exec_env->fragment_mgr()->cancel_instance(params.fragment_instance_id, + PPlanFragmentCancelReason::INTERNAL_ERROR); } void BackendService::transmit_data(TTransmitDataResult& return_val, @@ -648,8 +651,8 @@ void BackendService::ingest_binlog(TIngestBinlogResult& result, // Step 6.2: commit txn Status commit_txn_status = StorageEngine::instance()->txn_manager()->commit_txn( local_tablet->data_dir()->get_meta(), rowset_meta->partition_id(), - rowset_meta->txn_id(), rowset_meta->tablet_id(), rowset_meta->tablet_schema_hash(), - local_tablet->tablet_uid(), rowset_meta->load_id(), rowset, true); + rowset_meta->txn_id(), rowset_meta->tablet_id(), local_tablet->tablet_uid(), + rowset_meta->load_id(), rowset, true); if (!commit_txn_status && !commit_txn_status.is()) { auto err_msg = fmt::format( "failed to commit txn for remote tablet. rowset_id: {}, remote_tablet_id={}, " diff --git a/be/src/service/backend_service.h b/be/src/service/backend_service.h index 1cc952a0092734c..e98dd65a8c7a05c 100644 --- a/be/src/service/backend_service.h +++ b/be/src/service/backend_service.h @@ -68,7 +68,8 @@ class BackendService : public BackendServiceIf { ~BackendService() override = default; // NOTE: now we do not support multiple backend in one process - static Status create_service(ExecEnv* exec_env, int port, ThriftServer** server); + static Status create_service(ExecEnv* exec_env, int port, + std::unique_ptr* server); // Agent service void submit_tasks(TAgentResult& return_value, diff --git a/be/src/service/brpc_service.cpp b/be/src/service/brpc_service.cpp index 1c65d55ea8fc602..f35c49cbd29faa0 100644 --- a/be/src/service/brpc_service.cpp +++ b/be/src/service/brpc_service.cpp @@ -51,7 +51,9 @@ BRpcService::BRpcService(ExecEnv* exec_env) : _exec_env(exec_env), _server(new b : std::max((int64_t)1073741824, (MemInfo::mem_limit() / 1024) * 20); } -BRpcService::~BRpcService() = default; +BRpcService::~BRpcService() { + join(); +} Status BRpcService::start(int port, int num_threads) { // Add service diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index 033740d49b25394..ebdca8dec78e6ec 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -59,11 +59,14 @@ #include "olap/options.h" #include "olap/storage_engine.h" #include "runtime/exec_env.h" +#include "runtime/user_function_cache.h" #include "service/backend_options.h" #include "service/backend_service.h" #include "service/brpc_service.h" #include "service/http_service.h" #include "util/debug_util.h" +#include "util/disk_info.h" +#include "util/mem_info.h" #include "util/telemetry/telemetry.h" #include "util/thrift_rpc_helper.h" #include "util/thrift_server.h" @@ -80,8 +83,37 @@ void __lsan_do_leak_check(); } namespace doris { -extern bool k_doris_run; -extern bool k_doris_exit; + +void signal_handler(int signal) { + if (signal == SIGINT || signal == SIGTERM) { + k_doris_exit = true; + } +} + +int install_signal(int signo, void (*handler)(int)) { + struct sigaction sa; + memset(&sa, 0, sizeof(struct sigaction)); + sa.sa_handler = handler; + sigemptyset(&sa.sa_mask); + auto ret = sigaction(signo, &sa, nullptr); + if (ret != 0) { + char buf[64]; + LOG(ERROR) << "install signal failed, signo=" << signo << ", errno=" << errno + << ", errmsg=" << strerror_r(errno, buf, sizeof(buf)); + } + return ret; +} + +void init_signals() { + auto ret = install_signal(SIGINT, signal_handler); + if (ret < 0) { + exit(-1); + } + ret = install_signal(SIGTERM, signal_handler); + if (ret < 0) { + exit(-1); + } +} static void thrift_output(const char* x) { LOG(WARNING) << "thrift internal message: " << x; @@ -268,6 +300,7 @@ struct Checker { int main(int argc, char** argv) { doris::signal::InstallFailureSignalHandler(); + doris::init_signals(); // check if print version or help if (argc > 1) { @@ -331,6 +364,15 @@ int main(int argc, char** argv) { return -1; } + // ATTN: Callers that want to override default gflags variables should do so before calling this method + google::ParseCommandLineFlags(&argc, &argv, true); + // ATTN: MUST init before LOG + doris::init_glog("be"); + + LOG(INFO) << doris::get_version_string(false); + + doris::init_thrift_logging(); + if (doris::config::enable_fuzzy_mode) { LOG(INFO) << "enable_fuzzy_mode is true, set fuzzy configs"; doris::config::set_fuzzy_configs(); @@ -392,6 +434,7 @@ int main(int argc, char** argv) { } } + // Load file cache before starting up daemon threads to make sure StorageEngine is read. if (doris::config::enable_file_cache) { doris::io::IFileCache::init(); std::unordered_set cache_path_set; @@ -433,16 +476,26 @@ int main(int argc, char** argv) { } } + // ATTN: MUST init before `ExecEnv`, `StorageEngine` and other daemon services + // + // Daemon ───┬──► StorageEngine ──► ExecEnv ──► Disk/Mem/CpuInfo + // │ + // │ + // BackendService ─┘ + doris::CpuInfo::init(); + doris::DiskInfo::init(); + doris::MemInfo::init(); + doris::UserFunctionCache::instance()->init(doris::config::user_function_dir); + + LOG(INFO) << doris::CpuInfo::debug_string(); + LOG(INFO) << doris::DiskInfo::debug_string(); + LOG(INFO) << doris::MemInfo::debug_string(); + // PHDR speed up exception handling, but exceptions from dynamically loaded libraries (dlopen) // will work only after additional call of this function. // rewrites dl_iterate_phdr will cause Jemalloc to fail to run after enable profile. see # // updatePHDRCache(); - // Load file cache before starting up daemon threads to make sure StorageEngine is read. - doris::Daemon daemon; - daemon.init(argc, argv, paths); - daemon.start(); - doris::ResourceTls::init(); if (!doris::BackendOptions::init()) { exit(-1); @@ -454,8 +507,6 @@ int main(int argc, char** argv) { doris::TabletSchemaCache::create_global_schema_cache(); doris::vectorized::init_date_day_offset_dict(); - doris::k_doris_run = true; - // init s3 write buffer pool doris::io::S3FileBufferPool* s3_buffer_pool = doris::io::S3FileBufferPool::GetInstance(); s3_buffer_pool->init(doris::config::s3_write_buffer_whole_size, @@ -466,25 +517,27 @@ int main(int argc, char** argv) { doris::EngineOptions options; options.store_paths = paths; options.backend_uid = doris::UniqueId::gen_uid(); - doris::StorageEngine* engine = nullptr; + std::unique_ptr engine; auto st = doris::StorageEngine::open(options, &engine); if (!st.ok()) { LOG(FATAL) << "fail to open StorageEngine, res=" << st; exit(-1); } - exec_env->set_storage_engine(engine); engine->set_heartbeat_flags(exec_env->heartbeat_flags()); // start all background threads of storage engine. // SHOULD be called after exec env is initialized. EXIT_IF_ERROR(engine->start_bg_threads()); + doris::Daemon daemon; + daemon.start(); + doris::telemetry::init_tracer(); // begin to start services doris::ThriftRpcHelper::setup(exec_env); // 1. thrift server with be_port - doris::ThriftServer* be_server = nullptr; + std::unique_ptr be_server; EXIT_IF_ERROR( doris::BackendService::create_service(exec_env, doris::config::be_port, &be_server)); status = be_server->start(); @@ -515,7 +568,7 @@ int main(int argc, char** argv) { // 4. heart beat server doris::TMasterInfo* master_info = exec_env->master_info(); - doris::ThriftServer* heartbeat_thrift_server; + std::unique_ptr heartbeat_thrift_server; doris::Status heartbeat_status = doris::create_heartbeat_server( exec_env, doris::config::heartbeat_service_port, &heartbeat_thrift_server, doris::config::heartbeat_service_thread_count, master_info); @@ -537,29 +590,9 @@ int main(int argc, char** argv) { #if defined(LEAK_SANITIZER) __lsan_do_leak_check(); #endif - sleep(10); + sleep(3); } - doris::TabletSchemaCache::stop_and_join(); - http_service.stop(); - brpc_service.join(); - daemon.stop(); - heartbeat_thrift_server->stop(); - heartbeat_thrift_server->join(); - be_server->stop(); - be_server->join(); - engine->stop(); - - delete be_server; - be_server = nullptr; - - delete heartbeat_thrift_server; - heartbeat_thrift_server = nullptr; - - doris::ExecEnv::destroy(exec_env); - delete engine; - engine = nullptr; - return 0; } diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp index ed28b457a27c956..dcc30829729bc4d 100644 --- a/be/src/service/http_service.cpp +++ b/be/src/service/http_service.cpp @@ -31,6 +31,7 @@ #include "http/action/download_binlog_action.h" #include "http/action/file_cache_action.h" #include "http/action/health_action.h" +#include "http/action/http_stream.h" #include "http/action/jeprofile_actions.h" #include "http/action/meta_action.h" #include "http/action/metrics_action.h" @@ -42,7 +43,6 @@ #include "http/action/snapshot_action.h" #include "http/action/stream_load.h" #include "http/action/stream_load_2pc.h" -#include "http/action/stream_load_with_sql.h" #include "http/action/tablet_migration_action.h" #include "http/action/tablets_distribution_action.h" #include "http/action/tablets_info_action.h" @@ -63,7 +63,9 @@ HttpService::HttpService(ExecEnv* env, int port, int num_threads) _ev_http_server(new EvHttpServer(port, num_threads)), _web_page_handler(new WebPageHandler(_ev_http_server.get())) {} -HttpService::~HttpService() {} +HttpService::~HttpService() { + stop(); +} Status HttpService::start() { add_default_path_handlers(_web_page_handler.get()); @@ -80,11 +82,9 @@ Status HttpService::start() { _ev_http_server->register_handler(HttpMethod::PUT, "/api/{db}/{table}/_stream_load_2pc", streamload_2pc_action); - // register stream load with sql - StreamLoadWithSqlAction* streamload_with_sql_action = - _pool.add(new StreamLoadWithSqlAction(_env)); - _ev_http_server->register_handler(HttpMethod::PUT, "/api/_stream_load_with_sql", - streamload_with_sql_action); + // register http_stream + HttpStreamAction* http_stream_action = _pool.add(new HttpStreamAction(_env)); + _ev_http_server->register_handler(HttpMethod::PUT, "/api/_http_stream", http_stream_action); // register download action std::vector allow_paths; diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index b458af6336317e2..87916ed4c43c8e2 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -559,10 +559,11 @@ void PInternalServiceImpl::cancel_plan_fragment(google::protobuf::RpcController* if (request->has_cancel_reason()) { LOG(INFO) << "cancel fragment, fragment_instance_id=" << print_id(tid) << ", reason: " << PPlanFragmentCancelReason_Name(request->cancel_reason()); - _exec_env->fragment_mgr()->cancel(tid, request->cancel_reason()); + _exec_env->fragment_mgr()->cancel_instance(tid, request->cancel_reason()); } else { LOG(INFO) << "cancel fragment, fragment_instance_id=" << print_id(tid); - _exec_env->fragment_mgr()->cancel(tid); + _exec_env->fragment_mgr()->cancel_instance(tid, + PPlanFragmentCancelReason::INTERNAL_ERROR); } // TODO: the logic seems useless, cancel only return Status::OK. remove it st.to_protobuf(result->mutable_status()); @@ -753,7 +754,7 @@ void PInternalServiceImpl::_get_column_ids_by_tablet_ids( for (const auto& param : params) { int64_t index_id = param.indexid(); auto tablet_ids = param.tablet_ids(); - std::set> filter_set; + std::set> filter_set; for (const int64_t tablet_id : tablet_ids) { TabletSharedPtr tablet = tablet_mgr->get_tablet(tablet_id); if (tablet == nullptr) { @@ -766,9 +767,10 @@ void PInternalServiceImpl::_get_column_ids_by_tablet_ids( } // check schema consistency, column ids should be the same const auto& columns = tablet->tablet_schema()->columns(); - std::vector column_ids(columns.size()); - std::transform(columns.begin(), columns.end(), column_ids.begin(), - [](const TabletColumn& c) { return c.unique_id(); }); + std::set column_ids; + for (const auto& col : columns) { + column_ids.insert(col.unique_id()); + } filter_set.insert(column_ids); } if (filter_set.size() > 1) { @@ -1450,8 +1452,8 @@ void PInternalServiceImpl::request_slave_tablet_pull_rowset( } Status commit_txn_status = StorageEngine::instance()->txn_manager()->commit_txn( tablet->data_dir()->get_meta(), rowset_meta->partition_id(), rowset_meta->txn_id(), - rowset_meta->tablet_id(), rowset_meta->tablet_schema_hash(), tablet->tablet_uid(), - rowset_meta->load_id(), rowset, true); + rowset_meta->tablet_id(), tablet->tablet_uid(), rowset_meta->load_id(), rowset, + true); if (!commit_txn_status && !commit_txn_status.is()) { LOG(WARNING) << "failed to add committed rowset for slave replica. rowset_id=" << rowset_meta->rowset_id() << ", tablet_id=" << rowset_meta->tablet_id() @@ -1727,7 +1729,7 @@ void PInternalServiceImpl::get_tablet_rowset_versions(google::protobuf::RpcContr google::protobuf::Closure* done) { brpc::ClosureGuard closure_guard(done); VLOG_DEBUG << "receive get tablet versions request: " << request->DebugString(); - ExecEnv::GetInstance()->storage_engine()->get_tablet_rowset_versions(request, response); + StorageEngine::instance()->get_tablet_rowset_versions(request, response); } void PInternalServiceImpl::glob(google::protobuf::RpcController* controller, diff --git a/be/src/service/point_query_executor.cpp b/be/src/service/point_query_executor.cpp index 4463c0d01ab4c76..c7d1b523621f293 100644 --- a/be/src/service/point_query_executor.cpp +++ b/be/src/service/point_query_executor.cpp @@ -287,7 +287,7 @@ Status PointQueryExecutor::_lookup_row_key() { st = (_tablet->lookup_row_key(_row_read_ctxs[i]._primary_key, true, specified_rowsets, &location, INT32_MAX /*rethink?*/, segment_caches, rowset_ptr.get())); - if (st.is()) { + if (st.is()) { continue; } RETURN_IF_ERROR(st); diff --git a/be/src/service/point_query_executor.h b/be/src/service/point_query_executor.h index 58ddba35c98b62d..a49bfb442f933bf 100644 --- a/be/src/service/point_query_executor.h +++ b/be/src/service/point_query_executor.h @@ -198,8 +198,8 @@ class LookupConnectionCache : public LRUCachePolicy { private: friend class PointQueryExecutor; LookupConnectionCache(size_t capacity) - : LRUCachePolicy("LookupConnectionCache", capacity, LRUCacheType::SIZE, - config::tablet_lookup_cache_clean_interval) {} + : LRUCachePolicy(CachePolicy::CacheType::LOOKUP_CONNECTION_CACHE, capacity, + LRUCacheType::SIZE, config::tablet_lookup_cache_clean_interval) {} std::string encode_key(__int128_t cache_id) { fmt::memory_buffer buffer; diff --git a/be/src/util/debug_util.cpp b/be/src/util/debug_util.cpp index 9ea88acc25dd614..2d44c281a5d1df0 100644 --- a/be/src/util/debug_util.cpp +++ b/be/src/util/debug_util.cpp @@ -17,6 +17,7 @@ #include "util/debug_util.h" +#include #include #include @@ -26,6 +27,7 @@ #include #include "common/version_internal.h" +#include "util/uid_util.h" namespace doris { @@ -101,4 +103,35 @@ std::string hexdump(const char* buf, int len) { return ss.str(); } +std::string PrintThriftNetworkAddress(const TNetworkAddress& add) { + std::stringstream ss; + add.printTo(ss); + return ss.str(); +} + +std::string PrintFrontendInfos(const std::vector& fe_infos) { + std::stringstream ss; + const size_t count = fe_infos.size(); + + for (int i = 0; i < count; ++i) { + fe_infos[i].printTo(ss); + ss << ' '; + } + + return ss.str(); +} + +std::string PrintFrontendInfo(const TFrontendInfo& fe_info) { + std::stringstream ss; + fe_info.printTo(ss); + + return ss.str(); +} + +std::string PrintInstanceStandardInfo(const TUniqueId& qid, const int fid, const TUniqueId& iid) { + std::stringstream ss; + ss << print_id(iid) << '|' << fid << '|' << print_id(qid); + return ss.str(); +} + } // namespace doris diff --git a/be/src/util/debug_util.h b/be/src/util/debug_util.h index fbc0c221f42a51f..e6b6491b8a9302e 100644 --- a/be/src/util/debug_util.h +++ b/be/src/util/debug_util.h @@ -17,6 +17,7 @@ #pragma once +#include #include #include #include @@ -30,6 +31,14 @@ std::string print_tstmt_type(const TStmtType::type& type); std::string print_query_state(const QueryState::type& type); std::string PrintTUnit(const TUnit::type& type); std::string PrintTMetricKind(const TMetricKind::type& type); +std::string PrintThriftNetworkAddress(const TNetworkAddress&); +std::string PrintFrontendInfo(const TFrontendInfo& fe_info); +std::string PrintFrontendInfos(const std::vector& fe_infos); + +// A desirable scenario would be to call this function WHENEVER whenever we need to print instance information. +// By using a fixed format, we would be able to identify all the paths in which this instance is executed. +// InstanceId|FragmentIdx|QueryId +std::string PrintInstanceStandardInfo(const TUniqueId& qid, const int fid, const TUniqueId& iid); // Returns a string " (build )" // If compact == false, this string is appended: "\nBuilt on " diff --git a/be/src/util/proto_util.h b/be/src/util/proto_util.h index 3c583b867cc4a20..b11180ab290b4ec 100644 --- a/be/src/util/proto_util.h +++ b/be/src/util/proto_util.h @@ -68,7 +68,7 @@ void transmit_block(PBackendService_Stub& stub, Closure* closure, } template -Status transmit_block_http(RuntimeState* state, Closure* closure, PTransmitDataParams& params, +Status transmit_block_http(ExecEnv* exec_env, Closure* closure, PTransmitDataParams& params, TNetworkAddress brpc_dest_addr) { RETURN_IF_ERROR(request_embed_attachment_contain_block(¶ms, closure)); @@ -76,8 +76,7 @@ Status transmit_block_http(RuntimeState* state, Closure* closure, PTransmitDataP std::string brpc_url = get_brpc_http_url(brpc_dest_addr.hostname, brpc_dest_addr.port); std::shared_ptr brpc_http_stub = - state->exec_env()->brpc_internal_client_cache()->get_new_client_no_cache(brpc_url, - "http"); + exec_env->brpc_internal_client_cache()->get_new_client_no_cache(brpc_url, "http"); closure->cntl.http_request().uri() = brpc_url + "/PInternalServiceImpl/transmit_block_by_http"; closure->cntl.http_request().set_method(brpc::HTTP_METHOD_POST); closure->cntl.http_request().set_content_type("application/json"); diff --git a/be/src/util/stack_util.cpp b/be/src/util/stack_util.cpp index 01ced12708bb4cd..ca328089357aa7d 100644 --- a/be/src/util/stack_util.cpp +++ b/be/src/util/stack_util.cpp @@ -45,6 +45,9 @@ std::string get_stack_trace() { } else if (tool == "glibc") { return get_stack_trace_by_glibc(); } else if (tool == "libunwind") { +#if defined(__APPLE__) // TODO + return get_stack_trace_by_glog(); +#endif return get_stack_trace_by_libunwind(); } else { return "no stack"; diff --git a/be/src/util/thrift_server.cpp b/be/src/util/thrift_server.cpp index 88e71d16c7610fd..06e5996313057df 100644 --- a/be/src/util/thrift_server.cpp +++ b/be/src/util/thrift_server.cpp @@ -299,6 +299,11 @@ ThriftServer::ThriftServer(const std::string& name, INT_COUNTER_METRIC_REGISTER(_thrift_server_metric_entity, thrift_connections_total); } +ThriftServer::~ThriftServer() { + stop(); + join(); +} + Status ThriftServer::start() { DCHECK(!_started); std::shared_ptr protocol_factory( diff --git a/be/src/util/thrift_server.h b/be/src/util/thrift_server.h index 3603fbc468f5f91..08c09a6de521f51 100644 --- a/be/src/util/thrift_server.h +++ b/be/src/util/thrift_server.h @@ -80,7 +80,7 @@ class ThriftServer { int num_worker_threads = DEFAULT_WORKER_THREADS, ServerType server_type = THREADED); - ~ThriftServer() {} + ~ThriftServer(); int port() const { return _port; } diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h index 7d2634a8dcc50d4..b070bae8c1d5445 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h @@ -225,7 +225,7 @@ class AggregateFunctionBitmapSerializationHelper auto& col = assert_cast(*assert_cast(column)); auto* data = col.get_data().data(); for (size_t i = 0; i != num_rows; ++i) { - this->data(places[i]).merge(data[i]); + this->data(places[i] + offset).merge(data[i]); } } else { BaseHelper::deserialize_and_merge_vec(places, offset, rhs, column, arena, num_rows); @@ -240,7 +240,7 @@ class AggregateFunctionBitmapSerializationHelper auto* data = col.get_data().data(); for (size_t i = 0; i != num_rows; ++i) { if (places[i]) { - this->data(places[i]).merge(data[i]); + this->data(places[i] + offset).merge(data[i]); } } } else { diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h b/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h index d7b1fe72b9e1bda..382957302ec245f 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h @@ -190,7 +190,7 @@ class AggregateFunctionBitmapAgg final auto& col = assert_cast(*assert_cast(column)); auto* data = col.get_data().data(); for (size_t i = 0; i != num_rows; ++i) { - this->data(places[i]).value |= data[i]; + this->data(places[i] + offset).value |= data[i]; } } @@ -201,7 +201,7 @@ class AggregateFunctionBitmapAgg final auto* data = col.get_data().data(); for (size_t i = 0; i != num_rows; ++i) { if (places[i]) { - this->data(places[i]).value |= data[i]; + this->data(places[i] + offset).value |= data[i]; } } } diff --git a/be/src/vec/aggregate_functions/aggregate_function_sort.h b/be/src/vec/aggregate_functions/aggregate_function_sort.h index f82061e071a8f4f..39d7fd184f56884 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_sort.h +++ b/be/src/vec/aggregate_functions/aggregate_function_sort.h @@ -87,7 +87,8 @@ struct AggregateFunctionSortData { PBlock pblock; pblock.ParseFromString(data); - block = Block(pblock); + auto st = block.deserialize(pblock); + CHECK(st.ok()); } void add(const IColumn** columns, size_t columns_num, size_t row_num) { diff --git a/be/src/vec/columns/column_array.h b/be/src/vec/columns/column_array.h index 18bdc74bc11456a..b42800d5b2696a2 100644 --- a/be/src/vec/columns/column_array.h +++ b/be/src/vec/columns/column_array.h @@ -210,6 +210,10 @@ class ColumnArray final : public COWHelper { return scatter_impl(num_columns, selector); } + size_t ALWAYS_INLINE offset_at(ssize_t i) const { return get_offsets()[i - 1]; } + size_t ALWAYS_INLINE size_at(ssize_t i) const { + return get_offsets()[i] - get_offsets()[i - 1]; + } void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector) const override { return append_data_by_selector_impl(res, selector); @@ -258,11 +262,6 @@ class ColumnArray final : public COWHelper { WrappedPtr data; WrappedPtr offsets; - size_t ALWAYS_INLINE offset_at(ssize_t i) const { return get_offsets()[i - 1]; } - size_t ALWAYS_INLINE size_at(ssize_t i) const { - return get_offsets()[i] - get_offsets()[i - 1]; - } - /// Multiply values if the nested column is ColumnVector. template ColumnPtr replicate_number(const IColumn::Offsets& replicate_offsets) const; diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h index e89bc64868627bc..b6db5af7e9edab8 100644 --- a/be/src/vec/columns/column_dictionary.h +++ b/be/src/vec/columns/column_dictionary.h @@ -318,7 +318,16 @@ class ColumnDictionary final : public COWHelper> { if (is_dict_sorted() && !is_dict_code_converted()) { convert_dict_codes_if_necessary(); } - auto res = vectorized::PredicateColumnType::create(); + // if type is OLAP_FIELD_TYPE_CHAR, we need to construct TYPE_CHAR PredicateColumnType, + // because the string length will different from varchar and string which needed to be processed after. + auto create_column = [this]() -> MutableColumnPtr { + if (_type == FieldType::OLAP_FIELD_TYPE_CHAR) { + return vectorized::PredicateColumnType::create(); + } + return vectorized::PredicateColumnType::create(); + }; + + auto res = create_column(); res->reserve(_codes.capacity()); for (size_t i = 0; i < _codes.size(); ++i) { auto& code = reinterpret_cast(_codes[i]); diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index e5c7d7101485fe5..dab4c66d63a76da 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -288,7 +288,7 @@ class ColumnNullable final : public COWHelper { bool only_null() const override { return nested_column->is_dummy(); } // used in schema change - void swap_nested_column(ColumnPtr& other) { ((ColumnPtr&)nested_column).swap(other); } + void change_nested_column(ColumnPtr& other) { ((ColumnPtr&)nested_column) = other; } /// Return the column that represents values. IColumn& get_nested_column() { return *nested_column; } diff --git a/be/src/vec/columns/predicate_column.h b/be/src/vec/columns/predicate_column.h index fd7548c9f5d1441..21e503817cf3c36 100644 --- a/be/src/vec/columns/predicate_column.h +++ b/be/src/vec/columns/predicate_column.h @@ -43,12 +43,11 @@ class PredicateColumnType final : public COWHelper>; using T = typename PredicatePrimitiveTypeTraits::PredicateFieldType; using ColumnType = typename PrimitiveTypeTraits::ColumnType; - PredicateColumnType(const PredicateColumnType& src) : data(src.data.begin(), src.data.end()) {} - uint64_t get_date_at(uint16_t idx) { const T val = data[idx]; const char* val_ptr = reinterpret_cast(&val); @@ -154,8 +153,16 @@ class PredicateColumnType final : public COWHelper) { + auto res = reinterpret_cast(data[n]); + if constexpr (Type == TYPE_CHAR) { + res.size = strnlen(res.data, res.size); + } + return res; + } else { + LOG(FATAL) << "should not call get_data_at in predicate column except for string type"; + } } void insert_from(const IColumn& src, size_t n) override { diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index 8342ba98f29d62a..0fea95a90ec28df 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -84,7 +84,8 @@ Block::Block(const std::vector& slots, size_t block_size, } } -Block::Block(const PBlock& pblock) { +Status Block::deserialize(const PBlock& pblock) { + swap(Block()); int be_exec_version = pblock.has_be_exec_version() ? pblock.be_exec_version() : 0; CHECK(BeExecVersionManager::check_be_exec_version(be_exec_version)); @@ -98,11 +99,12 @@ Block::Block(const PBlock& pblock) { size_t uncompressed_size = 0; if (pblock.has_compression_type() && pblock.has_uncompressed_size()) { BlockCompressionCodec* codec; - get_block_compression_codec(pblock.compression_type(), &codec); + RETURN_IF_ERROR(get_block_compression_codec(pblock.compression_type(), &codec)); uncompressed_size = pblock.uncompressed_size(); compression_scratch.resize(uncompressed_size); Slice decompressed_slice(compression_scratch); - codec->decompress(Slice(compressed_data, compressed_size), &decompressed_slice); + RETURN_IF_ERROR(codec->decompress(Slice(compressed_data, compressed_size), + &decompressed_slice)); DCHECK(uncompressed_size == decompressed_slice.size); } else { bool success = snappy::GetUncompressedLength(compressed_data, compressed_size, @@ -126,6 +128,8 @@ Block::Block(const PBlock& pblock) { data.emplace_back(data_column->get_ptr(), type, pcol_meta.name()); } initialize_index_by_name(); + + return Status::OK(); } void Block::reserve(size_t count) { @@ -400,6 +404,15 @@ void Block::skip_num_rows(int64_t& length) { size_t Block::bytes() const { size_t res = 0; for (const auto& elem : data) { + if (!elem.column) { + std::stringstream ss; + for (const auto& e : data) { + ss << e.name + " "; + } + LOG(FATAL) << fmt::format( + "Column {} in block is nullptr, in method bytes. All Columns are {}", elem.name, + ss.str()); + } res += elem.column->byte_size(); } @@ -409,6 +422,15 @@ size_t Block::bytes() const { size_t Block::allocated_bytes() const { size_t res = 0; for (const auto& elem : data) { + if (!elem.column) { + std::stringstream ss; + for (const auto& e : data) { + ss << e.name + " "; + } + LOG(FATAL) << fmt::format( + "Column {} in block is nullptr, in method allocated_bytes. All Columns are {}", + elem.name, ss.str()); + } res += elem.column->allocated_bytes(); } diff --git a/be/src/vec/core/block.h b/be/src/vec/core/block.h index 900969bda351e60..cad45ac23791925 100644 --- a/be/src/vec/core/block.h +++ b/be/src/vec/core/block.h @@ -86,7 +86,6 @@ class Block { Block() = default; Block(std::initializer_list il); Block(const ColumnsWithTypeAndName& data_); - Block(const PBlock& pblock); Block(const std::vector& slots, size_t block_size, bool ignore_trivial_slot = false); @@ -308,6 +307,8 @@ class Block { size_t* compressed_bytes, segment_v2::CompressionTypePB compression_type, bool allow_transfer_large_data = false) const; + Status deserialize(const PBlock& pblock); + std::unique_ptr create_same_struct_block(size_t size) const; /** Compares (*this) n-th row and rhs m-th row. diff --git a/be/src/vec/core/block_spill_reader.cpp b/be/src/vec/core/block_spill_reader.cpp index 8d2d48122964054..d0cebd3043bf687 100644 --- a/be/src/vec/core/block_spill_reader.cpp +++ b/be/src/vec/core/block_spill_reader.cpp @@ -134,7 +134,8 @@ Status BlockSpillReader::read(Block* block, bool* eos) { if (!pb_block.ParseFromArray(result.data, result.size)) { return Status::InternalError("Failed to read spilled block"); } - new_block = Block::create_unique(pb_block); + new_block = Block::create_unique(); + RETURN_IF_ERROR(new_block->deserialize(pb_block)); } block->swap(*new_block); } else { diff --git a/be/src/vec/data_types/data_type_decimal.h b/be/src/vec/data_types/data_type_decimal.h index bc2ac4ab68fad95..bdd95988362584d 100644 --- a/be/src/vec/data_types/data_type_decimal.h +++ b/be/src/vec/data_types/data_type_decimal.h @@ -89,11 +89,6 @@ constexpr size_t max_decimal_precision() { return 38; } -template -constexpr typename T::NativeType max_decimal_value() { - return 0; -} - DataTypePtr create_decimal(UInt64 precision, UInt64 scale, bool use_v2); inline UInt32 least_decimal_precision_for(TypeIndex int_type) { @@ -577,4 +572,17 @@ ToDataType::FieldType convert_to_decimal(const typename FromDataType::FieldType& } } +template + requires IsDecimalNumber +typename T::NativeType max_decimal_value(UInt32 precision) { + return type_limit::max() / DataTypeDecimal::get_scale_multiplier( + (UInt32)(max_decimal_precision() - precision)); +} + +template + requires IsDecimalNumber +typename T::NativeType min_decimal_value(UInt32 precision) { + return type_limit::min() / DataTypeDecimal::get_scale_multiplier( + (UInt32)(max_decimal_precision() - precision)); +} } // namespace doris::vectorized 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 222564de0ecbbf8..46b9c851d39700b 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 @@ -52,12 +52,10 @@ class DataTypeArraySerDe : public DataTypeSerDe { const FormatOptions& options) const override; Status write_column_to_pb(const IColumn& column, PValues& result, int start, int end) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("write_column_to_pb with type " + column.get_name()); } Status read_column_from_pb(IColumn& column, const PValues& arg) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("read_column_from_pb with type " + column.get_name()); } void write_one_cell_to_jsonb(const IColumn& column, JsonbWriter& result, Arena* mem_pool, 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 3a36aad612e243e..8bf532b4dcec6dc 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 @@ -46,14 +46,14 @@ class DataTypeBitMapSerDe : public DataTypeSerDe { } Status deserialize_one_cell_from_text(IColumn& column, Slice& slice, const FormatOptions& options) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("deserialize_one_cell_from_text with type " + + column.get_name()); } Status deserialize_column_from_text_vector(IColumn& column, std::vector& slices, int* num_deserialized, const FormatOptions& options) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("deserialize_column_from_text_vector with type " + + column.get_name()); } Status write_column_to_pb(const IColumn& column, PValues& result, int start, @@ -68,12 +68,12 @@ class DataTypeBitMapSerDe : public DataTypeSerDe { arrow::ArrayBuilder* array_builder, int start, int end) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "write_column_to_arrow with type " + column.get_name()); } void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "read_column_from_arrow with type " + column.get_name()); } Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, 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 afabb2f71da513e..5c60e68895bf993 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 @@ -219,7 +219,8 @@ void DataTypeDate64SerDe::read_column_from_arrow(IColumn& column, const arrow::A const auto type = std::static_pointer_cast(arrow_array->type()); divisor = time_unit_divisor(type->unit()); if (divisor == 0L) { - LOG(FATAL) << "Invalid Time Type:" << type->name(); + throw doris::Exception(doris::ErrorCode::INVALID_ARGUMENT, + "Invalid Time Type: " + type->name()); } for (size_t value_i = start; value_i < end; ++value_i) { VecDateTimeValue v; 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 645ba928ec7ca1f..97d14078cd332bc 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 @@ -64,7 +64,7 @@ class DataTypeDateTimeV2SerDe : public DataTypeNumberSerDe { void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "read_column_from_arrow with type " + column.get_name()); } Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, 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 cc4ee16d944dd88..95bf1a4b1b42e36 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 @@ -149,7 +149,8 @@ void DataTypeDecimalSerDe::write_column_to_arrow(const IColumn& column, const array_builder->type()->name()); } } else { - LOG(FATAL) << "Not support write " << column.get_name() << " to arrow"; + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "write_column_to_arrow with type " + column.get_name()); } } @@ -190,7 +191,8 @@ void DataTypeDecimalSerDe::read_column_from_arrow(IColumn& column, column_data.emplace_back(*reinterpret_cast(concrete_array->Value(value_i))); } } else { - LOG(FATAL) << "Not support read " << column.get_name() << " from arrow"; + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "read_column_from_arrow with type " + column.get_name()); } } 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 f6826d6b448b4f8..57cf98327aba202 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 @@ -172,7 +172,8 @@ void DataTypeDecimalSerDe::write_one_cell_to_jsonb(const IColumn& column, Jso Decimal64::NativeType val = *reinterpret_cast(data_ref.data); result.writeInt64(val); } else { - LOG(FATAL) << "unknown Column " << column.get_name() << " for writing to jsonb"; + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "write_one_cell_to_jsonb with type " + column.get_name()); } } @@ -189,7 +190,8 @@ void DataTypeDecimalSerDe::read_one_cell_from_jsonb(IColumn& column, } else if constexpr (std::is_same_v>) { col.insert_value(static_cast(arg)->val()); } else { - LOG(FATAL) << "unknown jsonb " << arg->typeName() << " for writing to column"; + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "read_one_cell_from_jsonb with type " + column.get_name()); } } } // namespace vectorized 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 aa357aeb806984e..46928bdb99e2302 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 @@ -39,68 +39,64 @@ class DataTypeFixedLengthObjectSerDe : public DataTypeSerDe { void serialize_one_cell_to_text(const IColumn& column, int row_num, BufferWritable& bw, FormatOptions& options) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "serialize_one_cell_to_text with type " + column.get_name()); } void serialize_column_to_text(const IColumn& column, int start_idx, int end_idx, BufferWritable& bw, FormatOptions& options) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "serialize_column_to_text with type " + column.get_name()); } Status deserialize_one_cell_from_text(IColumn& column, Slice& slice, const FormatOptions& options) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("deserialize_one_cell_from_text with type " + + column.get_name()); } Status deserialize_column_from_text_vector(IColumn& column, std::vector& slices, int* num_deserialized, const FormatOptions& options) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("deserialize_column_from_text_vector with type " + + column.get_name()); } Status write_column_to_pb(const IColumn& column, PValues& result, int start, int end) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("write_column_to_pb with type " + column.get_name()); } Status read_column_from_pb(IColumn& column, const PValues& arg) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("read_column_from_pb with type " + column.get_name()); }; void write_one_cell_to_jsonb(const IColumn& column, JsonbWriter& result, Arena* mem_pool, int32_t col_id, int row_num) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "write_one_cell_to_jsonb with type " + column.get_name()); } void read_one_cell_from_jsonb(IColumn& column, const JsonbValue* arg) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "read_one_cell_from_jsonb with type " + column.get_name()); } void write_column_to_arrow(const IColumn& column, const NullMap* null_map, arrow::ArrayBuilder* array_builder, int start, int end) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "write_column_to_arrow with type " + column.get_name()); } void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "read_column_from_arrow with type " + column.get_name()); } Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("write_column_to_mysql with type " + column.get_name()); } Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("write_column_to_pb with type " + column.get_name()); } }; } // namespace vectorized 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 46f90fc20c0fa02..b7bae1302a9d687 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 @@ -56,7 +56,7 @@ class DataTypeHLLSerDe : public DataTypeSerDe { void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "read_column_from_arrow with type " + column.get_name()); } Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, 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 e90ba11f2994368..05c129114a1501e 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 @@ -51,12 +51,10 @@ class DataTypeMapSerDe : public DataTypeSerDe { const FormatOptions& options) const override; Status write_column_to_pb(const IColumn& column, PValues& result, int start, int end) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("write_column_to_pb with type " + column.get_name()); } Status read_column_from_pb(IColumn& column, const PValues& arg) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("read_column_from_pb with type " + column.get_name()); } void write_one_cell_to_jsonb(const IColumn& column, JsonbWriter& result, Arena* mem_pool, int32_t col_id, int row_num) const override; 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 1e27ef03d12c28f..cfe84a4f8fb7597 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 @@ -235,7 +235,8 @@ void DataTypeNumberSerDe::read_one_cell_from_jsonb(IColumn& column, } else if constexpr (std::is_same_v) { col.insert_value(static_cast(arg)->val()); } else { - LOG(FATAL) << "unknown jsonb type " << arg->typeName() << " for writing to column"; + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "read_one_cell_from_jsonb with type '{}'", arg->typeName()); } } template @@ -271,7 +272,8 @@ void DataTypeNumberSerDe::write_one_cell_to_jsonb(const IColumn& column, double val = *reinterpret_cast(data_ref.data); result.writeDouble(val); } else { - LOG(FATAL) << "unknown column type " << column.get_name() << " for writing to jsonb"; + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "write_one_cell_to_jsonb with type " + column.get_name()); } } 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 3dddc06113dcbba..dc421767797df19 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 @@ -39,68 +39,64 @@ class DataTypeObjectSerDe : public DataTypeSerDe { void serialize_one_cell_to_text(const IColumn& column, int row_num, BufferWritable& bw, FormatOptions& options) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "serialize_one_cell_to_text with type " + column.get_name()); } void serialize_column_to_text(const IColumn& column, int start_idx, int end_idx, BufferWritable& bw, FormatOptions& options) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "serialize_column_to_text with type " + column.get_name()); } Status deserialize_one_cell_from_text(IColumn& column, Slice& slice, const FormatOptions& options) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("deserialize_one_cell_from_text with type " + + column.get_name()); } Status deserialize_column_from_text_vector(IColumn& column, std::vector& slices, int* num_deserialized, const FormatOptions& options) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("deserialize_column_from_text_vector with type " + + column.get_name()); } Status write_column_to_pb(const IColumn& column, PValues& result, int start, int end) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("write_column_to_pb with type " + column.get_name()); } Status read_column_from_pb(IColumn& column, const PValues& arg) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("read_column_from_pb with type " + column.get_name()); } void write_one_cell_to_jsonb(const IColumn& column, JsonbWriter& result, Arena* mem_pool, int32_t col_id, int row_num) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "write_one_cell_to_jsonb with type " + column.get_name()); } void read_one_cell_from_jsonb(IColumn& column, const JsonbValue* arg) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "read_one_cell_from_jsonb with type " + column.get_name()); } void write_column_to_arrow(const IColumn& column, const NullMap* null_map, arrow::ArrayBuilder* array_builder, int start, int end) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "write_column_to_arrow with type " + column.get_name()); } void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "read_column_from_arrow with type " + column.get_name()); } Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("write_column_to_mysql with type " + column.get_name()); } Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, int row_idx, bool col_const) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("write_column_to_mysql with type " + column.get_name()); } }; } // namespace vectorized 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 4c9dae672beee0b..e9fe7a8a5d59b3a 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 @@ -43,25 +43,25 @@ class DataTypeQuantileStateSerDe : public DataTypeSerDe { void serialize_one_cell_to_text(const IColumn& column, int row_num, BufferWritable& bw, FormatOptions& options) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "serialize_one_cell_to_text with type " + column.get_name()); } void serialize_column_to_text(const IColumn& column, int start_idx, int end_idx, BufferWritable& bw, FormatOptions& options) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "serialize_column_to_text with type " + column.get_name()); } Status deserialize_one_cell_from_text(IColumn& column, Slice& slice, const FormatOptions& options) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("deserialize_one_cell_from_text with type " + + column.get_name()); } Status deserialize_column_from_text_vector(IColumn& column, std::vector& slices, int* num_deserialized, const FormatOptions& options) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("deserialize_column_from_text_vector with type " + + column.get_name()); } Status write_column_to_pb(const IColumn& column, PValues& result, int start, @@ -76,12 +76,12 @@ class DataTypeQuantileStateSerDe : public DataTypeSerDe { arrow::ArrayBuilder* array_builder, int start, int end) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "write_column_to_arrow with type " + column.get_name()); } void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + "read_column_from_arrow with type " + column.get_name()); } Status write_column_to_mysql(const IColumn& column, MysqlRowBuffer& row_buffer, 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 33d14c74115dfba..ec36e12ebbf8a2d 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 @@ -53,25 +53,22 @@ class DataTypeStructSerDe : public DataTypeSerDe { Status deserialize_one_cell_from_text(IColumn& column, Slice& slice, const FormatOptions& options) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "deserialize_one_cell_from_text with type " + column.get_name()); + return Status::NotSupported("deserialize_one_cell_from_text with type " + + column.get_name()); } Status deserialize_column_from_text_vector(IColumn& column, std::vector& slices, int* num_deserialized, const FormatOptions& options) const override { - throw doris::Exception( - ErrorCode::NOT_IMPLEMENTED_ERROR, - "deserialize_column_from_text_vector with type " + column.get_name()); + return Status::NotSupported("deserialize_column_from_text_vector with type " + + column.get_name()); } Status write_column_to_pb(const IColumn& column, PValues& result, int start, int end) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_pb with type " + column.get_name()); + return Status::NotSupported("write_column_to_pb with type " + column.get_name()); } Status read_column_from_pb(IColumn& column, const PValues& arg) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "read_column_from_pb with type " + column.get_name()); + return Status::NotSupported("read_column_from_pb with type " + column.get_name()); } void write_one_cell_to_jsonb(const IColumn& column, JsonbWriter& result, Arena* mem_pool, int32_t col_id, int row_num) const override; diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_reader.cpp index e627a96e0d22d24..49798ed4f13f729 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_reader.cpp @@ -206,10 +206,6 @@ void ParquetReader::_close_internal() { } _closed = true; } - - if (_is_file_metadata_owned && _file_metadata != nullptr) { - delete _file_metadata; - } } Status ParquetReader::_open_file() { @@ -230,23 +226,25 @@ Status ParquetReader::_open_file() { } size_t meta_size = 0; if (_meta_cache == nullptr) { - _is_file_metadata_owned = true; RETURN_IF_ERROR( parse_thrift_footer(_file_reader, &_file_metadata, &meta_size, _io_ctx)); + // wrap it with unique ptr, so that it can be released finally. + _file_metadata_ptr.reset(_file_metadata); + _file_metadata = _file_metadata_ptr.get(); + _column_statistics.read_bytes += meta_size; // parse magic number & parse meta data _column_statistics.meta_read_calls += 1; } else { - _is_file_metadata_owned = false; - RETURN_IF_ERROR(_meta_cache->get_parquet_footer(_file_reader, _io_ctx, 0, &meta_size, - &_cache_handle)); - + RETURN_IF_ERROR(_meta_cache->get_parquet_footer(_file_reader, _io_ctx, + _file_description.mtime, &meta_size, + &_meta_cache_handle)); _column_statistics.read_bytes += meta_size; if (meta_size > 0) { _column_statistics.meta_read_calls += 1; } - _file_metadata = (FileMetaData*)_cache_handle.data(); + _file_metadata = (FileMetaData*)_meta_cache_handle.data(); } if (_file_metadata == nullptr) { diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.h b/be/src/vec/exec/format/parquet/vparquet_reader.h index b2f8a2bd19efd54..5ceca55d7ecd24c 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_reader.h @@ -214,14 +214,21 @@ class ParquetReader : public GenericReader { const TFileRangeDesc& _scan_range; io::FileSystemProperties _system_properties; io::FileDescription _file_description; - std::shared_ptr _file_system = nullptr; - io::FileReaderSPtr _file_reader = nullptr; - ObjLRUCache::CacheHandle _cache_handle; + + // the following fields are for parquet meta data cache. + // if _meta_cache is not null, the _file_metadata will be got from _meta_cache, + // and it is owned by _meta_cache_handle. + // if _meta_cache is null, _file_metadata will be managed by _file_metadata_ptr, + // which will be released when deconstructing. + // ATTN: these fields must be before _file_reader, to make sure they will be released + // after _file_reader. Otherwise, there may be heap-use-after-free bug. + ObjLRUCache::CacheHandle _meta_cache_handle; + std::unique_ptr _file_metadata_ptr; FileMetaData* _file_metadata = nullptr; - // set to true if _file_metadata is owned by this reader. - // otherwise, it is owned by someone else, such as _meta_cache - bool _is_file_metadata_owned = false; const tparquet::FileMetaData* _t_metadata; + + std::shared_ptr _file_system = nullptr; + io::FileReaderSPtr _file_reader = nullptr; std::unique_ptr _current_group_reader = nullptr; // read to the end of current reader bool _row_group_eof = true; diff --git a/be/src/vec/exec/join/process_hash_table_probe.h b/be/src/vec/exec/join/process_hash_table_probe.h index 79e126584c3ff05..3b189506ef046fd 100644 --- a/be/src/vec/exec/join/process_hash_table_probe.h +++ b/be/src/vec/exec/join/process_hash_table_probe.h @@ -29,7 +29,7 @@ namespace vectorized { class Block; class MutableBlock; -class HashJoinNode; +struct HashJoinProbeContext; using MutableColumnPtr = IColumn::MutablePtr; using MutableColumns = std::vector; @@ -39,7 +39,8 @@ using ConstNullMapPtr = const NullMap*; template struct ProcessHashTableProbe { - ProcessHashTableProbe(HashJoinNode* join_node, int batch_size); + ProcessHashTableProbe(HashJoinProbeContext* join_context, int batch_size); + ~ProcessHashTableProbe() = default; // output build side result column template @@ -82,7 +83,7 @@ struct ProcessHashTableProbe { Status process_data_in_hashtable(HashTableType& hash_table_ctx, MutableBlock& mutable_block, Block* output_block, bool* eos); - vectorized::HashJoinNode* _join_node; + vectorized::HashJoinProbeContext* _join_context; const int _batch_size; const std::vector& _build_blocks; std::unique_ptr _arena; diff --git a/be/src/vec/exec/join/process_hash_table_probe_impl.h b/be/src/vec/exec/join/process_hash_table_probe_impl.h index 8c4a22a0146c74f..8bef813bf0cbad5 100644 --- a/be/src/vec/exec/join/process_hash_table_probe_impl.h +++ b/be/src/vec/exec/join/process_hash_table_probe_impl.h @@ -26,29 +26,29 @@ namespace doris::vectorized { -static constexpr int PREFETCH_STEP = HashJoinNode::PREFETCH_STEP; - template -ProcessHashTableProbe::ProcessHashTableProbe(HashJoinNode* join_node, int batch_size) - : _join_node(join_node), +ProcessHashTableProbe::ProcessHashTableProbe(HashJoinProbeContext* join_context, + int batch_size) + : _join_context(join_context), _batch_size(batch_size), - _build_blocks(*join_node->_build_blocks), - _tuple_is_null_left_flags(join_node->_is_outer_join - ? &(reinterpret_cast( - *join_node->_tuple_is_null_left_flag_column) - .get_data()) - : nullptr), + _build_blocks(*join_context->_build_blocks), + _tuple_is_null_left_flags( + join_context->_is_outer_join + ? &(reinterpret_cast( + **join_context->_tuple_is_null_left_flag_column) + .get_data()) + : nullptr), _tuple_is_null_right_flags( - join_node->_is_outer_join + join_context->_is_outer_join ? &(reinterpret_cast( - *join_node->_tuple_is_null_right_flag_column) + **join_context->_tuple_is_null_right_flag_column) .get_data()) : nullptr), - _rows_returned_counter(join_node->_rows_returned_counter), - _search_hashtable_timer(join_node->_search_hashtable_timer), - _build_side_output_timer(join_node->_build_side_output_timer), - _probe_side_output_timer(join_node->_probe_side_output_timer), - _probe_process_hashtable_timer(join_node->_probe_process_hashtable_timer) {} + _rows_returned_counter(join_context->_rows_returned_counter), + _search_hashtable_timer(join_context->_search_hashtable_timer), + _build_side_output_timer(join_context->_build_side_output_timer), + _probe_side_output_timer(join_context->_probe_side_output_timer), + _probe_process_hashtable_timer(join_context->_probe_process_hashtable_timer) {} template template @@ -129,7 +129,7 @@ void ProcessHashTableProbe::probe_side_output_column( MutableColumns& mcol, const std::vector& output_slot_flags, int size, int last_probe_index, size_t probe_size, bool all_match_one, bool have_other_join_conjunct) { - auto& probe_block = _join_node->_probe_block; + auto& probe_block = *_join_context->_probe_block; for (int i = 0; i < output_slot_flags.size(); ++i) { if (output_slot_flags[i]) { auto& column = probe_block.get_by_position(i).column; @@ -180,18 +180,18 @@ void ProcessHashTableProbe::_pre_serialize_key( serialized_keys[i] = serialize_keys_to_pool_contiguous(i, keys_size, key_columns, *_arena); } - _join_node->_probe_arena_memory_usage->add(_arena->size() - old_probe_keys_memory_usage); + _join_context->_probe_arena_memory_usage->add(_arena->size() - old_probe_keys_memory_usage); } else { if (!_serialize_key_arena) { _serialize_key_arena.reset(new Arena); } if (total_bytes > _serialized_key_buffer_size) { - _join_node->_probe_arena_memory_usage->add(-_serialized_key_buffer_size); + _join_context->_probe_arena_memory_usage->add(-_serialized_key_buffer_size); _serialized_key_buffer_size = total_bytes; _serialize_key_arena->clear(); _serialized_key_buffer = reinterpret_cast( _serialize_key_arena->alloc(_serialized_key_buffer_size)); - _join_node->_probe_arena_memory_usage->add(_serialized_key_buffer_size); + _join_context->_probe_arena_memory_usage->add(_serialized_key_buffer_size); } for (size_t i = 0; i < key_rows; ++i) { @@ -213,8 +213,8 @@ Status ProcessHashTableProbe::do_process(HashTableType& hash_table_c MutableBlock& mutable_block, Block* output_block, size_t probe_rows, bool is_mark_join) { - auto& probe_index = _join_node->_probe_index; - auto& probe_raw_ptrs = _join_node->_probe_columns; + auto& probe_index = *_join_context->_probe_index; + auto& probe_raw_ptrs = *_join_context->_probe_columns; _probe_indexs.resize(_batch_size); if (_build_block_rows.size() < probe_rows * PROBE_SIDE_EXPLODE_RATE) { @@ -225,10 +225,10 @@ Status ProcessHashTableProbe::do_process(HashTableType& hash_table_c using Mapped = typename HashTableType::Mapped; int right_col_idx = - _join_node->_is_right_semi_anti ? 0 : _join_node->_left_table_data_types.size(); - int right_col_len = _join_node->_right_table_data_types.size(); + _join_context->_is_right_semi_anti ? 0 : _join_context->_left_table_data_types->size(); + int right_col_len = _join_context->_right_table_data_types->size(); - KeyGetter key_getter(probe_raw_ptrs, _join_node->_probe_key_sz, nullptr); + KeyGetter key_getter(probe_raw_ptrs, _join_context->_probe_key_sz, nullptr); if constexpr (ColumnsHashing::IsPreSerializedKeysHashMethodTraits::value) { if (probe_index == 0) { @@ -250,7 +250,7 @@ Status ProcessHashTableProbe::do_process(HashTableType& hash_table_c int last_probe_index = probe_index; size_t probe_size = 0; auto& probe_row_match_iter = - std::get>(_join_node->_probe_row_match_iter); + std::get>(*_join_context->_probe_row_match_iter); { SCOPED_TIMER(_search_hashtable_timer); if constexpr (!is_right_semi_anti_join) { @@ -435,15 +435,15 @@ Status ProcessHashTableProbe::do_process(HashTableType& hash_table_c { SCOPED_TIMER(_build_side_output_timer); build_side_output_column(mcol, right_col_idx, right_col_len, - _join_node->_right_output_slot_flags, current_offset); + *_join_context->_right_output_slot_flags, current_offset); } if constexpr (JoinOpType != TJoinOp::RIGHT_SEMI_JOIN && JoinOpType != TJoinOp::RIGHT_ANTI_JOIN) { SCOPED_TIMER(_probe_side_output_timer); - RETURN_IF_CATCH_EXCEPTION( - probe_side_output_column(mcol, _join_node->_left_output_slot_flags, current_offset, - last_probe_index, probe_size, all_match_one, false)); + RETURN_IF_CATCH_EXCEPTION(probe_side_output_column( + mcol, *_join_context->_left_output_slot_flags, current_offset, last_probe_index, + probe_size, all_match_one, false)); } output_block->swap(mutable_block.to_block()); @@ -456,8 +456,8 @@ template ::do_process_with_other_join_conjuncts( HashTableType& hash_table_ctx, ConstNullMapPtr null_map, MutableBlock& mutable_block, Block* output_block, size_t probe_rows, bool is_mark_join) { - auto& probe_index = _join_node->_probe_index; - auto& probe_raw_ptrs = _join_node->_probe_columns; + auto& probe_index = *_join_context->_probe_index; + auto& probe_raw_ptrs = *_join_context->_probe_columns; if (_build_block_rows.size() < probe_rows * PROBE_SIDE_EXPLODE_RATE) { _probe_indexs.resize(probe_rows * PROBE_SIDE_EXPLODE_RATE); _build_block_rows.resize(probe_rows * PROBE_SIDE_EXPLODE_RATE); @@ -469,7 +469,7 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( if constexpr (std::is_same_v) { constexpr auto probe_all = JoinOpType == TJoinOp::LEFT_OUTER_JOIN || JoinOpType == TJoinOp::FULL_OUTER_JOIN; - KeyGetter key_getter(probe_raw_ptrs, _join_node->_probe_key_sz, nullptr); + KeyGetter key_getter(probe_raw_ptrs, _join_context->_probe_key_sz, nullptr); if constexpr (ColumnsHashing::IsPreSerializedKeysHashMethodTraits::value) { if (probe_index == 0) { @@ -478,8 +478,8 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( key_getter.set_serialized_keys(_probe_keys.data()); } - int right_col_idx = _join_node->_left_table_data_types.size(); - int right_col_len = _join_node->_right_table_data_types.size(); + int right_col_idx = _join_context->_left_table_data_types->size(); + int right_col_len = _join_context->_right_table_data_types->size(); auto& mcol = mutable_block.mutable_columns(); // use in right join to change visited state after @@ -499,7 +499,7 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( bool is_the_last_sub_block = false; size_t probe_size = 0; auto& probe_row_match_iter = - std::get>(_join_node->_probe_row_match_iter); + std::get>(*_join_context->_probe_row_match_iter); if (probe_row_match_iter.ok()) { SCOPED_TIMER(_search_hashtable_timer); auto origin_offset = current_offset; @@ -703,12 +703,13 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( { SCOPED_TIMER(_build_side_output_timer); build_side_output_column(mcol, right_col_idx, right_col_len, - _join_node->_right_output_slot_flags, current_offset); + *_join_context->_right_output_slot_flags, + current_offset); } { SCOPED_TIMER(_probe_side_output_timer); RETURN_IF_CATCH_EXCEPTION(probe_side_output_column( - mcol, _join_node->_left_output_slot_flags, current_offset, last_probe_index, + mcol, *_join_context->_left_output_slot_flags, current_offset, last_probe_index, probe_size, all_match_one, true)); } auto num_cols = mutable_block.columns(); @@ -717,12 +718,12 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( // dispose the other join conjunct exec auto row_count = output_block->rows(); if (row_count) { - SCOPED_TIMER(_join_node->_process_other_join_conjunct_timer); + SCOPED_TIMER(_join_context->_process_other_join_conjunct_timer); int orig_columns = output_block->columns(); IColumn::Filter other_conjunct_filter(row_count, 1); bool can_be_filter_all; RETURN_IF_ERROR(VExprContext::execute_conjuncts( - _join_node->_other_join_conjuncts, nullptr, output_block, + *_join_context->_other_join_conjuncts, nullptr, output_block, &other_conjunct_filter, &can_be_filter_all)); auto result_column_id = output_block->columns(); @@ -748,7 +749,7 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( null_map_data, filter_map, output_block); // This is the last sub block of splitted block, and no equal-conjuncts-matched tuple // is output in all sub blocks, need to output a tuple for this probe row - if (is_the_last_sub_block && !_join_node->_is_any_probe_match_row_output) { + if (is_the_last_sub_block && !*_join_context->_is_any_probe_match_row_output) { filter_map[0] = true; null_map_data[0] = true; } @@ -799,7 +800,7 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( // It contains the first sub block of splited equal-conjuncts-matched tuples of the current probe row if (multi_matched_output_row_count > 0) { - _join_node->_is_any_probe_match_row_output = false; + *_join_context->_is_any_probe_match_row_output = false; _process_splited_equal_matched_tuples( row_count - multi_matched_output_row_count, multi_matched_output_row_count, column, visited_map, right_col_idx, @@ -821,7 +822,7 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( size_t start_row_idx = 1; // We are handling euqual-conjuncts matched tuples that are splitted into multiple blocks if (row_count_from_last_probe > 0) { - if (_join_node->_is_any_probe_match_row_output) { + if (*_join_context->_is_any_probe_match_row_output) { // if any matched tuple for this probe row is output, // ignore all the following tuples for this probe row. for (int row_idx = 0; row_idx < row_count_from_last_probe; ++row_idx) { @@ -850,14 +851,14 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( if (multi_matched_output_row_count > 0) { // If a matched row is output, all the equal-matched tuples in // the following sub blocks should be ignored - _join_node->_is_any_probe_match_row_output = filter_map[row_count - 1]; + *_join_context->_is_any_probe_match_row_output = filter_map[row_count - 1]; } else if (row_count_from_last_probe > 0 && - !_join_node->_is_any_probe_match_row_output) { + !*_join_context->_is_any_probe_match_row_output) { // We are handling euqual-conjuncts matched tuples that are splitted into multiple blocks, // and no matched tuple has been output in all previous run. // If a tuple is output in this run, all the following mathced tuples should be ignored if (filter_map[row_count_from_last_probe - 1]) { - _join_node->_is_any_probe_match_row_output = true; + *_join_context->_is_any_probe_match_row_output = true; } } @@ -898,7 +899,7 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( size_t start_row_idx = 1; // We are handling euqual-conjuncts matched tuples that are splitted into multiple blocks if (row_count_from_last_probe > 0) { - if (_join_node->_is_any_probe_match_row_output) { + if (*_join_context->_is_any_probe_match_row_output) { // if any matched tuple for this probe row is output, // ignore all the following tuples for this probe row. for (int row_idx = 0; row_idx < row_count_from_last_probe; ++row_idx) { @@ -951,16 +952,16 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( int end_row_idx; if (row_count_from_last_probe > 0) { end_row_idx = row_count - multi_matched_output_row_count; - if (!_join_node->_is_any_probe_match_row_output) { + if (!*_join_context->_is_any_probe_match_row_output) { // We are handling euqual-conjuncts matched tuples that are splitted into multiple blocks, // and no matched tuple has been output in all previous run. // If a tuple is output in this run, all the following mathced tuples should be ignored if (filter_map[row_count_from_last_probe - 1]) { - _join_node->_is_any_probe_match_row_output = true; + *_join_context->_is_any_probe_match_row_output = true; filter_map[row_count_from_last_probe - 1] = false; } if (is_the_last_sub_block && - !_join_node->_is_any_probe_match_row_output) { + !*_join_context->_is_any_probe_match_row_output) { // This is the last sub block of splitted block, and no equal-conjuncts-matched tuple // is output in all sub blocks, output a tuple for this probe row filter_map[0] = true; @@ -970,7 +971,8 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( // It contains the first sub block of splited equal-conjuncts-matched tuples of the current probe row // If a matched row is output, all the equal-matched tuples in // the following sub blocks should be ignored - _join_node->_is_any_probe_match_row_output = filter_map[row_count - 1]; + *_join_context->_is_any_probe_match_row_output = + filter_map[row_count - 1]; filter_map[row_count - 1] = false; } } else if (multi_matched_output_row_count > 0) { @@ -978,7 +980,7 @@ Status ProcessHashTableProbe::do_process_with_other_join_conjuncts( // It contains the first sub block of splited equal-conjuncts-matched tuples of the current probe row // If a matched row is output, all the equal-matched tuples in // the following sub blocks should be ignored - _join_node->_is_any_probe_match_row_output = filter_map[row_count - 1]; + *_join_context->_is_any_probe_match_row_output = filter_map[row_count - 1]; filter_map[row_count - 1] = false; } else { end_row_idx = row_count; @@ -1075,7 +1077,7 @@ void ProcessHashTableProbe::_process_splited_equal_matched_tuples( *visited_map[i] |= other_hit; } } - _join_node->_is_any_probe_match_row_output |= + *_join_context->_is_any_probe_match_row_output |= simd::contain_byte(filter_map + start_row_idx, row_count, 1); } @@ -1093,15 +1095,15 @@ Status ProcessHashTableProbe::process_data_in_hashtable(HashTableTyp auto& mcol = mutable_block.mutable_columns(); bool right_semi_anti_without_other = - _join_node->_is_right_semi_anti && !_join_node->_have_other_join_conjunct; + _join_context->_is_right_semi_anti && !_join_context->_have_other_join_conjunct; int right_col_idx = - right_semi_anti_without_other ? 0 : _join_node->_left_table_data_types.size(); - int right_col_len = _join_node->_right_table_data_types.size(); + right_semi_anti_without_other ? 0 : _join_context->_left_table_data_types->size(); + int right_col_len = _join_context->_right_table_data_types->size(); auto& iter = hash_table_ctx.iter; auto block_size = 0; auto& visited_iter = - std::get>(_join_node->_outer_join_pull_visited_iter); + std::get>(*_join_context->_outer_join_pull_visited_iter); _build_blocks_locs.resize(_batch_size); auto register_build_loc = [&](int8_t offset, int32_t row_nums) { _build_blocks_locs[block_size++] = std::pair(offset, row_nums); @@ -1212,7 +1214,7 @@ Status ProcessHashTableProbe::process_data_in_hashtable(HashTableTyp } // just resize the left table column in case with other conjunct to make block size is not zero - if (_join_node->_is_right_semi_anti && _join_node->_have_other_join_conjunct) { + if (_join_context->_is_right_semi_anti && _join_context->_have_other_join_conjunct) { auto target_size = mcol[right_col_idx]->size(); for (int i = 0; i < right_col_idx; ++i) { mcol[i]->resize(target_size); diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp index 7aaddf4b2aa20ab..55985d6fed8ef17 100644 --- a/be/src/vec/exec/join/vhash_join_node.cpp +++ b/be/src/vec/exec/join/vhash_join_node.cpp @@ -43,6 +43,8 @@ #include "exprs/runtime_filter.h" #include "exprs/runtime_filter_slots.h" #include "gutil/strings/substitute.h" +#include "pipeline/exec/hashjoin_build_sink.h" +#include "pipeline/exec/hashjoin_probe_operator.h" #include "runtime/define_primitive_type.h" #include "runtime/descriptors.h" #include "runtime/query_context.h" @@ -72,8 +74,6 @@ namespace doris::vectorized { -static constexpr int PREFETCH_STEP = HashJoinNode::PREFETCH_STEP; - template Status HashJoinNode::_extract_join_column( Block&, COW::mutable_ptr>&, std::vector>&, @@ -84,233 +84,141 @@ template Status HashJoinNode::_extract_join_column( std::vector>&, std::vector> const&); -using ProfileCounter = RuntimeProfile::Counter; - -template -struct Overload : Callables... { - using Callables::operator()...; -}; - -template -Overload(Callables&&... callables) -> Overload; +RuntimeFilterContext::RuntimeFilterContext(HashJoinNode* join_node) + : _runtime_filter_descs(join_node->_runtime_filter_descs), + _runtime_filter_slots(join_node->_runtime_filter_slots), + _build_expr_ctxs(join_node->_build_expr_ctxs), + _build_bf_cardinality(join_node->_build_bf_cardinality), + _inserted_rows(join_node->_inserted_rows), + _push_down_timer(join_node->_push_down_timer), + _push_compute_timer(join_node->_push_compute_timer) {} + +RuntimeFilterContext::RuntimeFilterContext(pipeline::HashJoinBuildSinkLocalState* local_state) + : _runtime_filter_descs(local_state->join_build()->_runtime_filter_descs), + _runtime_filter_slots(local_state->_runtime_filter_slots), + _build_expr_ctxs(local_state->_build_expr_ctxs), + _build_bf_cardinality(local_state->_build_bf_cardinality), + _inserted_rows(local_state->_inserted_rows), + _push_down_timer(local_state->_push_down_timer), + _push_compute_timer(local_state->_push_compute_timer) {} + +HashJoinProbeContext::HashJoinProbeContext(HashJoinNode* join_node) + : _have_other_join_conjunct(join_node->_have_other_join_conjunct), + _is_right_semi_anti(join_node->_is_right_semi_anti), + _is_outer_join(join_node->_is_outer_join), + _tuple_is_null_left_flag_column(&join_node->_tuple_is_null_left_flag_column), + _tuple_is_null_right_flag_column(&join_node->_tuple_is_null_right_flag_column), + _other_join_conjuncts(&join_node->_other_join_conjuncts), + _right_table_data_types(&join_node->_right_table_data_types), + _left_table_data_types(&join_node->_left_table_data_types), + _search_hashtable_timer(join_node->_search_hashtable_timer), + _build_side_output_timer(join_node->_build_side_output_timer), + _probe_side_output_timer(join_node->_probe_side_output_timer), + _probe_process_hashtable_timer(join_node->_probe_process_hashtable_timer), + _process_other_join_conjunct_timer(join_node->_process_other_join_conjunct_timer), + _rows_returned_counter(join_node->_rows_returned_counter), + _probe_arena_memory_usage(join_node->_probe_arena_memory_usage), + _arena(join_node->_arena), + _outer_join_pull_visited_iter(&join_node->_outer_join_pull_visited_iter), + _probe_row_match_iter(&join_node->_probe_row_match_iter), + _build_blocks(join_node->_build_blocks), + _probe_block(&join_node->_probe_block), + _probe_columns(&join_node->_probe_columns), + _probe_index(&join_node->_probe_index), + _probe_key_sz(join_node->_probe_key_sz), + _left_output_slot_flags(&join_node->_left_output_slot_flags), + _right_output_slot_flags(&join_node->_right_output_slot_flags), + _is_any_probe_match_row_output(&join_node->_is_any_probe_match_row_output) {} + +HashJoinProbeContext::HashJoinProbeContext(pipeline::HashJoinProbeLocalState* local_state) + : _have_other_join_conjunct(local_state->join_probe()->_have_other_join_conjunct), + _is_right_semi_anti(local_state->join_probe()->_is_right_semi_anti), + _is_outer_join(local_state->join_probe()->_is_outer_join), + _tuple_is_null_left_flag_column(&local_state->_tuple_is_null_left_flag_column), + _tuple_is_null_right_flag_column(&local_state->_tuple_is_null_right_flag_column), + _other_join_conjuncts(&local_state->_other_join_conjuncts), + _right_table_data_types(&local_state->join_probe()->_right_table_data_types), + _left_table_data_types(&local_state->join_probe()->_left_table_data_types), + _search_hashtable_timer(local_state->_search_hashtable_timer), + _build_side_output_timer(local_state->_build_side_output_timer), + _probe_side_output_timer(local_state->_probe_side_output_timer), + _probe_process_hashtable_timer(local_state->_probe_process_hashtable_timer), + _process_other_join_conjunct_timer(local_state->_process_other_join_conjunct_timer), + _rows_returned_counter(local_state->_rows_returned_counter), + _probe_arena_memory_usage(local_state->_probe_arena_memory_usage), + _arena(local_state->_shared_state->arena), + _outer_join_pull_visited_iter(&local_state->_shared_state->outer_join_pull_visited_iter), + _probe_row_match_iter(&local_state->_shared_state->probe_row_match_iter), + _build_blocks(local_state->_shared_state->build_blocks), + _probe_block(&local_state->_probe_block), + _probe_columns(&local_state->_probe_columns), + _probe_index(&local_state->_probe_index), + _probe_key_sz(local_state->_shared_state->probe_key_sz), + _left_output_slot_flags(&local_state->join_probe()->_left_output_slot_flags), + _right_output_slot_flags(&local_state->join_probe()->_right_output_slot_flags), + _is_any_probe_match_row_output(&local_state->_is_any_probe_match_row_output) {} + +HashJoinBuildContext::HashJoinBuildContext(HashJoinNode* join_node) + : _hash_table_memory_usage(join_node->_hash_table_memory_usage), + _build_buckets_counter(join_node->_build_buckets_counter), + _build_collisions_counter(join_node->_build_collisions_counter), + _build_buckets_fill_counter(join_node->_build_buckets_fill_counter), + _build_table_insert_timer(join_node->_build_table_insert_timer), + _build_table_expanse_timer(join_node->_build_table_expanse_timer), + _build_table_convert_timer(join_node->_build_table_convert_timer), + _build_side_compute_hash_timer(join_node->_build_side_compute_hash_timer), + _build_arena_memory_usage(join_node->_build_arena_memory_usage), + _profile(join_node->runtime_profile()), + _build_key_sz(join_node->_build_key_sz), + _build_unique(join_node->_build_unique), + _runtime_filter_descs(join_node->_runtime_filter_descs), + _inserted_rows(join_node->_inserted_rows), + _arena(join_node->_arena), + _build_bf_cardinality(join_node->_build_bf_cardinality) {} + +HashJoinBuildContext::HashJoinBuildContext(pipeline::HashJoinBuildSinkLocalState* local_state) + : _hash_table_memory_usage(local_state->_hash_table_memory_usage), + _build_buckets_counter(local_state->_build_buckets_counter), + _build_collisions_counter(local_state->_build_collisions_counter), + _build_buckets_fill_counter(local_state->_build_buckets_fill_counter), + _build_table_insert_timer(local_state->_build_table_insert_timer), + _build_table_expanse_timer(local_state->_build_table_expanse_timer), + _build_table_convert_timer(local_state->_build_table_convert_timer), + _build_side_compute_hash_timer(local_state->_build_side_compute_hash_timer), + _build_arena_memory_usage(local_state->_build_arena_memory_usage), + _profile(local_state->profile()), + _build_key_sz(local_state->join_build()->_build_key_sz), + _build_unique(local_state->join_build()->_build_unique), + _runtime_filter_descs(local_state->join_build()->_runtime_filter_descs), + _inserted_rows(local_state->_inserted_rows), + _arena(local_state->_shared_state->arena), + _build_bf_cardinality(local_state->_build_bf_cardinality) {} template -struct ProcessHashTableBuild { - ProcessHashTableBuild(int rows, Block& acquired_block, ColumnRawPtrs& build_raw_ptrs, - HashJoinNode* join_node, int batch_size, uint8_t offset, - RuntimeState* state) - : _rows(rows), - _skip_rows(0), - _acquired_block(acquired_block), - _build_raw_ptrs(build_raw_ptrs), - _join_node(join_node), - _batch_size(batch_size), - _offset(offset), - _state(state), - _build_side_compute_hash_timer(join_node->_build_side_compute_hash_timer) {} - - template - Status run(HashTableContext& hash_table_ctx, ConstNullMapPtr null_map, bool* has_null_key) { - using KeyGetter = typename HashTableContext::State; - using Mapped = typename HashTableContext::Mapped; - - Defer defer {[&]() { - int64_t bucket_size = hash_table_ctx.hash_table.get_buffer_size_in_cells(); - int64_t filled_bucket_size = hash_table_ctx.hash_table.size(); - int64_t bucket_bytes = hash_table_ctx.hash_table.get_buffer_size_in_bytes(); - COUNTER_SET(_join_node->_hash_table_memory_usage, bucket_bytes); - COUNTER_SET(_join_node->_build_buckets_counter, bucket_size); - COUNTER_SET(_join_node->_build_collisions_counter, - hash_table_ctx.hash_table.get_collisions()); - COUNTER_SET(_join_node->_build_buckets_fill_counter, filled_bucket_size); - - auto hash_table_buckets = hash_table_ctx.hash_table.get_buffer_sizes_in_cells(); - std::string hash_table_buckets_info; - for (auto bucket_count : hash_table_buckets) { - hash_table_buckets_info += std::to_string(bucket_count) + ", "; - } - _join_node->add_hash_buckets_info(hash_table_buckets_info); - - auto hash_table_sizes = hash_table_ctx.hash_table.sizes(); - hash_table_buckets_info.clear(); - for (auto table_size : hash_table_sizes) { - hash_table_buckets_info += std::to_string(table_size) + ", "; - } - _join_node->add_hash_buckets_filled_info(hash_table_buckets_info); - }}; - - KeyGetter key_getter(_build_raw_ptrs, _join_node->_build_key_sz, nullptr); - - SCOPED_TIMER(_join_node->_build_table_insert_timer); - hash_table_ctx.hash_table.reset_resize_timer(); - - // only not build_unique, we need expanse hash table before insert data - // 1. There are fewer duplicate keys, reducing the number of resize hash tables - // can improve performance to a certain extent, about 2%-5% - // 2. There are many duplicate keys, and the hash table filled bucket is far less than - // the hash table build bucket, which may waste a lot of memory. - // TODO, use the NDV expansion of the key column in the optimizer statistics - if (!_join_node->_build_unique) { - RETURN_IF_CATCH_EXCEPTION(hash_table_ctx.hash_table.expanse_for_add_elem( - std::min(_rows, config::hash_table_pre_expanse_max_rows))); - } - - vector& inserted_rows = _join_node->_inserted_rows[&_acquired_block]; - bool has_runtime_filter = !_join_node->_runtime_filter_descs.empty(); - if (has_runtime_filter) { - inserted_rows.reserve(_batch_size); - } - - _build_side_hash_values.resize(_rows); - auto& arena = *(_join_node->_arena); - auto old_build_arena_memory = arena.size(); - { - SCOPED_TIMER(_build_side_compute_hash_timer); - if constexpr (ColumnsHashing::IsPreSerializedKeysHashMethodTraits::value) { - auto old_keys_memory = hash_table_ctx.keys_memory_usage; - hash_table_ctx.serialize_keys(_build_raw_ptrs, _rows); - key_getter.set_serialized_keys(hash_table_ctx.keys.data()); - _join_node->_build_arena_memory_usage->add(hash_table_ctx.keys_memory_usage - - old_keys_memory); - } - - for (size_t k = 0; k < _rows; ++k) { - if (k % 65536 == 0) { - RETURN_IF_CANCELLED(_state); - } - if constexpr (ignore_null) { - if ((*null_map)[k]) { - continue; - } - } - // If apply short circuit strategy for null value (e.g. join operator is - // NULL_AWARE_LEFT_ANTI_JOIN), we build hash table until we meet a null value. - if constexpr (short_circuit_for_null) { - if ((*null_map)[k]) { - DCHECK(has_null_key); - *has_null_key = true; - return Status::OK(); - } - } - if constexpr (ColumnsHashing::IsPreSerializedKeysHashMethodTraits< - KeyGetter>::value) { - _build_side_hash_values[k] = - hash_table_ctx.hash_table.hash(key_getter.get_key_holder(k, arena).key); - } else { - _build_side_hash_values[k] = - hash_table_ctx.hash_table.hash(key_getter.get_key_holder(k, arena)); - } - } - } - - bool build_unique = _join_node->_build_unique; -#define EMPLACE_IMPL(stmt) \ - for (size_t k = 0; k < _rows; ++k) { \ - if (k % 65536 == 0) { \ - RETURN_IF_CANCELLED(_state); \ - } \ - if constexpr (ignore_null) { \ - if ((*null_map)[k]) { \ - continue; \ - } \ - } \ - auto emplace_result = key_getter.emplace_key(hash_table_ctx.hash_table, \ - _build_side_hash_values[k], k, arena); \ - if (k + PREFETCH_STEP < _rows) { \ - key_getter.template prefetch_by_hash( \ - hash_table_ctx.hash_table, _build_side_hash_values[k + PREFETCH_STEP]); \ - } \ - stmt; \ - } - - if (has_runtime_filter && build_unique) { - EMPLACE_IMPL( - if (emplace_result.is_inserted()) { - new (&emplace_result.get_mapped()) Mapped({k, _offset}); - inserted_rows.push_back(k); - _join_node->_build_bf_cardinality++; - } else { _skip_rows++; }); - } else if (has_runtime_filter && !build_unique) { - EMPLACE_IMPL( - if (emplace_result.is_inserted()) { - new (&emplace_result.get_mapped()) Mapped({k, _offset}); - inserted_rows.push_back(k); - _join_node->_build_bf_cardinality++; - } else { - emplace_result.get_mapped().insert({k, _offset}, *(_join_node->_arena)); - inserted_rows.push_back(k); - }); - } else if (!has_runtime_filter && build_unique) { - EMPLACE_IMPL( - if (emplace_result.is_inserted()) { - new (&emplace_result.get_mapped()) Mapped({k, _offset}); - } else { _skip_rows++; }); - } else { - EMPLACE_IMPL( - if (emplace_result.is_inserted()) { - new (&emplace_result.get_mapped()) Mapped({k, _offset}); - } else { - emplace_result.get_mapped().insert({k, _offset}, *(_join_node->_arena)); - }); - } -#undef EMPLACE_IMPL - - _join_node->_build_arena_memory_usage->add(arena.size() - old_build_arena_memory); - - COUNTER_UPDATE(_join_node->_build_table_expanse_timer, - hash_table_ctx.hash_table.get_resize_timer_value()); - COUNTER_UPDATE(_join_node->_build_table_convert_timer, - hash_table_ctx.hash_table.get_convert_timer_value()); - +Status ProcessRuntimeFilterBuild::operator()(RuntimeState* state, + HashTableContext& hash_table_ctx) { + if (_context->_runtime_filter_descs.empty()) { return Status::OK(); } + _context->_runtime_filter_slots = std::make_shared( + _context->_build_expr_ctxs, _context->_runtime_filter_descs); -private: - const int _rows; - int _skip_rows; - Block& _acquired_block; - ColumnRawPtrs& _build_raw_ptrs; - HashJoinNode* _join_node; - int _batch_size; - uint8_t _offset; - RuntimeState* _state; - - ProfileCounter* _build_side_compute_hash_timer; - std::vector _build_side_hash_values; -}; - -template -struct ProcessRuntimeFilterBuild { - ProcessRuntimeFilterBuild(HashJoinNode* join_node) : _join_node(join_node) {} - - Status operator()(RuntimeState* state, HashTableContext& hash_table_ctx) { - if (_join_node->_runtime_filter_descs.empty()) { - return Status::OK(); - } - _join_node->_runtime_filter_slots = std::make_shared( - _join_node->_probe_expr_ctxs, _join_node->_build_expr_ctxs, - _join_node->_runtime_filter_descs); - - RETURN_IF_ERROR(_join_node->_runtime_filter_slots->init( - state, hash_table_ctx.hash_table.get_size(), _join_node->_build_bf_cardinality)); + RETURN_IF_ERROR(_context->_runtime_filter_slots->init( + state, hash_table_ctx.hash_table.get_size(), _context->_build_bf_cardinality)); - if (!_join_node->_runtime_filter_slots->empty() && !_join_node->_inserted_rows.empty()) { - { - SCOPED_TIMER(_join_node->_push_compute_timer); - _join_node->_runtime_filter_slots->insert(_join_node->_inserted_rows); - } - } + if (!_context->_runtime_filter_slots->empty() && !_context->_inserted_rows.empty()) { { - SCOPED_TIMER(_join_node->_push_down_timer); - RETURN_IF_ERROR(_join_node->_runtime_filter_slots->publish()); + SCOPED_TIMER(_context->_push_compute_timer); + _context->_runtime_filter_slots->insert(_context->_inserted_rows); } - - return Status::OK(); + } + { + SCOPED_TIMER(_context->_push_down_timer); + RETURN_IF_ERROR(_context->_runtime_filter_slots->publish()); } -private: - HashJoinNode* _join_node; -}; + return Status::OK(); +} HashJoinNode::HashJoinNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : VJoinNodeBase(pool, tnode, descs), @@ -331,7 +239,7 @@ HashJoinNode::HashJoinNode(ObjectPool* pool, const TPlanNode& tnode, const Descr // avoid vector expand change block address. // one block can store 4g data, _build_blocks can store 128*4g data. // if probe data bigger than 512g, runtime filter maybe will core dump when insert data. - _build_blocks->reserve(_MAX_BUILD_BLOCK_COUNT); + _build_blocks->reserve(HASH_JOIN_MAX_BUILD_BLOCK_COUNT); } Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { @@ -896,10 +804,10 @@ Status HashJoinNode::sink(doris::RuntimeState* state, vectorized::Block* in_bloc } if (UNLIKELY(_build_side_mem_used - _build_side_last_mem_used > BUILD_BLOCK_MAX_SIZE)) { - if (_build_blocks->size() == _MAX_BUILD_BLOCK_COUNT) { - return Status::NotSupported( - strings::Substitute("data size of right table in hash join > $0", - BUILD_BLOCK_MAX_SIZE * _MAX_BUILD_BLOCK_COUNT)); + if (_build_blocks->size() == HASH_JOIN_MAX_BUILD_BLOCK_COUNT) { + return Status::NotSupported(strings::Substitute( + "data size of right table in hash join > $0", + BUILD_BLOCK_MAX_SIZE * HASH_JOIN_MAX_BUILD_BLOCK_COUNT)); } _build_blocks->emplace_back(_build_side_mutable_block.to_block()); @@ -918,10 +826,10 @@ Status HashJoinNode::sink(doris::RuntimeState* state, vectorized::Block* in_bloc if (_should_build_hash_table && eos) { if (!_build_side_mutable_block.empty()) { - if (_build_blocks->size() == _MAX_BUILD_BLOCK_COUNT) { - return Status::NotSupported( - strings::Substitute("data size of right table in hash join > $0", - BUILD_BLOCK_MAX_SIZE * _MAX_BUILD_BLOCK_COUNT)); + if (_build_blocks->size() == HASH_JOIN_MAX_BUILD_BLOCK_COUNT) { + return Status::NotSupported(strings::Substitute( + "data size of right table in hash join > $0", + BUILD_BLOCK_MAX_SIZE * HASH_JOIN_MAX_BUILD_BLOCK_COUNT)); } _build_blocks->emplace_back(_build_side_mutable_block.to_block()); COUNTER_UPDATE(_build_blocks_memory_usage, (*_build_blocks)[_build_block_idx].bytes()); @@ -934,8 +842,9 @@ Status HashJoinNode::sink(doris::RuntimeState* state, vectorized::Block* in_bloc }, [&](auto&& arg) -> Status { using HashTableCtxType = std::decay_t; + RuntimeFilterContext context(this); ProcessRuntimeFilterBuild - runtime_filter_build_process(this); + runtime_filter_build_process(&context); return runtime_filter_build_process(state, arg); }}, *_hash_table_variants); @@ -988,8 +897,7 @@ Status HashJoinNode::sink(doris::RuntimeState* state, vectorized::Block* in_bloc return Status::OK(); } _runtime_filter_slots = std::make_shared( - _probe_expr_ctxs, _build_expr_ctxs, - _runtime_filter_descs); + _build_expr_ctxs, _runtime_filter_descs); RETURN_IF_ERROR(_runtime_filter_slots->init( state, arg.hash_table.get_size(), 0)); @@ -1149,8 +1057,10 @@ Status HashJoinNode::_process_build_block(RuntimeState* state, Block& block, uin [&](auto&& arg, auto has_null_value, auto short_circuit_for_null_in_build_side) -> Status { using HashTableCtxType = std::decay_t; + HashJoinBuildContext context(this); ProcessHashTableBuild hash_table_build_process( - rows, block, raw_ptrs, this, state->batch_size(), offset, state); + rows, block, raw_ptrs, &context, state->batch_size(), offset, + state); return hash_table_build_process .template run( arg, @@ -1305,8 +1215,9 @@ void HashJoinNode::_process_hashtable_ctx_variants_init(RuntimeState* state) { std::visit( [&](auto&& join_op_variants) { using JoinOpType = std::decay_t; + _probe_context.reset(new HashJoinProbeContext(this)); _process_hashtable_ctx_variants->emplace>( - this, state->batch_size()); + _probe_context.get(), state->batch_size()); }, _join_op_variants); } diff --git a/be/src/vec/exec/join/vhash_join_node.h b/be/src/vec/exec/join/vhash_join_node.h index 85e53dbdab29d96..28cf86765d0ea17 100644 --- a/be/src/vec/exec/join/vhash_join_node.h +++ b/be/src/vec/exec/join/vhash_join_node.h @@ -58,12 +58,260 @@ class IRuntimeFilter; class DescriptorTbl; class RuntimeState; +namespace pipeline { +class HashJoinProbeLocalState; +class HashJoinBuildSinkLocalState; +} // namespace pipeline + namespace vectorized { struct UInt128; struct UInt256; template struct ProcessHashTableProbe; +class HashJoinNode; + +struct RuntimeFilterContext { + RuntimeFilterContext(HashJoinNode* join_node); + RuntimeFilterContext(pipeline::HashJoinBuildSinkLocalState* local_state); + std::vector& _runtime_filter_descs; + std::shared_ptr& _runtime_filter_slots; + VExprContextSPtrs& _build_expr_ctxs; + size_t& _build_bf_cardinality; + std::unordered_map>& _inserted_rows; + RuntimeProfile::Counter* _push_down_timer; + RuntimeProfile::Counter* _push_compute_timer; +}; + +template +struct ProcessRuntimeFilterBuild { + ProcessRuntimeFilterBuild(RuntimeFilterContext* context) : _context(context) {} + + Status operator()(RuntimeState* state, HashTableContext& hash_table_ctx); + +private: + RuntimeFilterContext* _context; +}; + +struct HashJoinBuildContext { + HashJoinBuildContext(HashJoinNode* join_node); + HashJoinBuildContext(pipeline::HashJoinBuildSinkLocalState* local_state); + + void add_hash_buckets_info(const std::string& info) { + _profile->add_info_string("HashTableBuckets", info); + } + void add_hash_buckets_filled_info(const std::string& info) { + _profile->add_info_string("HashTableFilledBuckets", info); + } + RuntimeProfile::Counter* _hash_table_memory_usage; + RuntimeProfile::Counter* _build_buckets_counter; + RuntimeProfile::Counter* _build_collisions_counter; + RuntimeProfile::Counter* _build_buckets_fill_counter; + RuntimeProfile::Counter* _build_table_insert_timer; + RuntimeProfile::Counter* _build_table_expanse_timer; + RuntimeProfile::Counter* _build_table_convert_timer; + RuntimeProfile::Counter* _build_side_compute_hash_timer; + RuntimeProfile::HighWaterMarkCounter* _build_arena_memory_usage; + RuntimeProfile* _profile; + + Sizes& _build_key_sz; + bool& _build_unique; + std::vector& _runtime_filter_descs; + std::unordered_map>& _inserted_rows; + std::shared_ptr& _arena; + size_t& _build_bf_cardinality; +}; + +using ProfileCounter = RuntimeProfile::Counter; + +// TODO: Best prefetch step is decided by machine. We should also provide a +// SQL hint to allow users to tune by hand. +static constexpr int PREFETCH_STEP = 64; + +template +struct ProcessHashTableBuild { + ProcessHashTableBuild(int rows, Block& acquired_block, ColumnRawPtrs& build_raw_ptrs, + HashJoinBuildContext* join_context, int batch_size, uint8_t offset, + RuntimeState* state) + : _rows(rows), + _skip_rows(0), + _acquired_block(acquired_block), + _build_raw_ptrs(build_raw_ptrs), + _join_context(join_context), + _batch_size(batch_size), + _offset(offset), + _state(state), + _build_side_compute_hash_timer(join_context->_build_side_compute_hash_timer) {} + + template + Status run(HashTableContext& hash_table_ctx, ConstNullMapPtr null_map, bool* has_null_key) { + using KeyGetter = typename HashTableContext::State; + using Mapped = typename HashTableContext::Mapped; + + Defer defer {[&]() { + int64_t bucket_size = hash_table_ctx.hash_table.get_buffer_size_in_cells(); + int64_t filled_bucket_size = hash_table_ctx.hash_table.size(); + int64_t bucket_bytes = hash_table_ctx.hash_table.get_buffer_size_in_bytes(); + COUNTER_SET(_join_context->_hash_table_memory_usage, bucket_bytes); + COUNTER_SET(_join_context->_build_buckets_counter, bucket_size); + COUNTER_SET(_join_context->_build_collisions_counter, + hash_table_ctx.hash_table.get_collisions()); + COUNTER_SET(_join_context->_build_buckets_fill_counter, filled_bucket_size); + + auto hash_table_buckets = hash_table_ctx.hash_table.get_buffer_sizes_in_cells(); + std::string hash_table_buckets_info; + for (auto bucket_count : hash_table_buckets) { + hash_table_buckets_info += std::to_string(bucket_count) + ", "; + } + _join_context->add_hash_buckets_info(hash_table_buckets_info); + + auto hash_table_sizes = hash_table_ctx.hash_table.sizes(); + hash_table_buckets_info.clear(); + for (auto table_size : hash_table_sizes) { + hash_table_buckets_info += std::to_string(table_size) + ", "; + } + _join_context->add_hash_buckets_filled_info(hash_table_buckets_info); + }}; + + KeyGetter key_getter(_build_raw_ptrs, _join_context->_build_key_sz, nullptr); + + SCOPED_TIMER(_join_context->_build_table_insert_timer); + hash_table_ctx.hash_table.reset_resize_timer(); + + // only not build_unique, we need expanse hash table before insert data + // 1. There are fewer duplicate keys, reducing the number of resize hash tables + // can improve performance to a certain extent, about 2%-5% + // 2. There are many duplicate keys, and the hash table filled bucket is far less than + // the hash table build bucket, which may waste a lot of memory. + // TODO, use the NDV expansion of the key column in the optimizer statistics + if (!_join_context->_build_unique) { + RETURN_IF_CATCH_EXCEPTION(hash_table_ctx.hash_table.expanse_for_add_elem( + std::min(_rows, config::hash_table_pre_expanse_max_rows))); + } + + vector& inserted_rows = _join_context->_inserted_rows[&_acquired_block]; + bool has_runtime_filter = !_join_context->_runtime_filter_descs.empty(); + if (has_runtime_filter) { + inserted_rows.reserve(_batch_size); + } + + _build_side_hash_values.resize(_rows); + auto& arena = *(_join_context->_arena); + auto old_build_arena_memory = arena.size(); + { + SCOPED_TIMER(_build_side_compute_hash_timer); + if constexpr (ColumnsHashing::IsPreSerializedKeysHashMethodTraits::value) { + auto old_keys_memory = hash_table_ctx.keys_memory_usage; + hash_table_ctx.serialize_keys(_build_raw_ptrs, _rows); + key_getter.set_serialized_keys(hash_table_ctx.keys.data()); + _join_context->_build_arena_memory_usage->add(hash_table_ctx.keys_memory_usage - + old_keys_memory); + } + + for (size_t k = 0; k < _rows; ++k) { + if (k % 65536 == 0) { + RETURN_IF_CANCELLED(_state); + } + if constexpr (ignore_null) { + if ((*null_map)[k]) { + continue; + } + } + // If apply short circuit strategy for null value (e.g. join operator is + // NULL_AWARE_LEFT_ANTI_JOIN), we build hash table until we meet a null value. + if constexpr (short_circuit_for_null) { + if ((*null_map)[k]) { + DCHECK(has_null_key); + *has_null_key = true; + return Status::OK(); + } + } + if constexpr (ColumnsHashing::IsPreSerializedKeysHashMethodTraits< + KeyGetter>::value) { + _build_side_hash_values[k] = + hash_table_ctx.hash_table.hash(key_getter.get_key_holder(k, arena).key); + } else { + _build_side_hash_values[k] = + hash_table_ctx.hash_table.hash(key_getter.get_key_holder(k, arena)); + } + } + } + + bool build_unique = _join_context->_build_unique; +#define EMPLACE_IMPL(stmt) \ + for (size_t k = 0; k < _rows; ++k) { \ + if (k % 65536 == 0) { \ + RETURN_IF_CANCELLED(_state); \ + } \ + if constexpr (ignore_null) { \ + if ((*null_map)[k]) { \ + continue; \ + } \ + } \ + auto emplace_result = key_getter.emplace_key(hash_table_ctx.hash_table, \ + _build_side_hash_values[k], k, arena); \ + if (k + PREFETCH_STEP < _rows) { \ + key_getter.template prefetch_by_hash( \ + hash_table_ctx.hash_table, _build_side_hash_values[k + PREFETCH_STEP]); \ + } \ + stmt; \ + } + + if (has_runtime_filter && build_unique) { + EMPLACE_IMPL( + if (emplace_result.is_inserted()) { + new (&emplace_result.get_mapped()) Mapped({k, _offset}); + inserted_rows.push_back(k); + _join_context->_build_bf_cardinality++; + } else { _skip_rows++; }); + } else if (has_runtime_filter && !build_unique) { + EMPLACE_IMPL( + if (emplace_result.is_inserted()) { + new (&emplace_result.get_mapped()) Mapped({k, _offset}); + inserted_rows.push_back(k); + _join_context->_build_bf_cardinality++; + } else { + emplace_result.get_mapped().insert({k, _offset}, *(_join_context->_arena)); + inserted_rows.push_back(k); + }); + } else if (!has_runtime_filter && build_unique) { + EMPLACE_IMPL( + if (emplace_result.is_inserted()) { + new (&emplace_result.get_mapped()) Mapped({k, _offset}); + } else { _skip_rows++; }); + } else { + EMPLACE_IMPL( + if (emplace_result.is_inserted()) { + new (&emplace_result.get_mapped()) Mapped({k, _offset}); + } else { + emplace_result.get_mapped().insert({k, _offset}, *(_join_context->_arena)); + }); + } +#undef EMPLACE_IMPL + + _join_context->_build_arena_memory_usage->add(arena.size() - old_build_arena_memory); + + COUNTER_UPDATE(_join_context->_build_table_expanse_timer, + hash_table_ctx.hash_table.get_resize_timer_value()); + COUNTER_UPDATE(_join_context->_build_table_convert_timer, + hash_table_ctx.hash_table.get_convert_timer_value()); + + return Status::OK(); + } + +private: + const int _rows; + int _skip_rows; + Block& _acquired_block; + ColumnRawPtrs& _build_raw_ptrs; + HashJoinBuildContext* _join_context; + int _batch_size; + uint8_t _offset; + RuntimeState* _state; + + ProfileCounter* _build_side_compute_hash_timer; + std::vector _build_side_hash_values; +}; template struct SerializedHashTableContext { @@ -219,12 +467,54 @@ using HashTableIteratorVariants = std::variant, ForwardIterator, ForwardIterator>; +static constexpr auto HASH_JOIN_MAX_BUILD_BLOCK_COUNT = 128; + +struct HashJoinProbeContext { + HashJoinProbeContext(HashJoinNode* join_node); + HashJoinProbeContext(pipeline::HashJoinProbeLocalState* local_state); + bool _have_other_join_conjunct; + const bool _is_right_semi_anti; + const bool _is_outer_join; + + MutableColumnPtr* _tuple_is_null_left_flag_column; + MutableColumnPtr* _tuple_is_null_right_flag_column; + + // other expr + VExprContextSPtrs* _other_join_conjuncts; + + DataTypes* _right_table_data_types; + DataTypes* _left_table_data_types; + + RuntimeProfile::Counter* _search_hashtable_timer; + RuntimeProfile::Counter* _build_side_output_timer; + RuntimeProfile::Counter* _probe_side_output_timer; + RuntimeProfile::Counter* _probe_process_hashtable_timer; + RuntimeProfile::Counter* _process_other_join_conjunct_timer; + RuntimeProfile::Counter* _rows_returned_counter; + RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage; + + std::shared_ptr _arena; + + // for full/right outer join + HashTableIteratorVariants* _outer_join_pull_visited_iter; + HashTableIteratorVariants* _probe_row_match_iter; + + std::shared_ptr> _build_blocks; + Block* _probe_block; + ColumnRawPtrs* _probe_columns; + int* _probe_index; + + Sizes _probe_key_sz; + + std::vector* _left_output_slot_flags; + std::vector* _right_output_slot_flags; + + // for cases when a probe row matches more than batch size build rows. + bool* _is_any_probe_match_row_output; +}; + class HashJoinNode final : public VJoinNodeBase { public: - // TODO: Best prefetch step is decided by machine. We should also provide a - // SQL hint to allow users to tune by hand. - static constexpr int PREFETCH_STEP = 64; - HashJoinNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); ~HashJoinNode() override; @@ -266,6 +556,10 @@ class HashJoinNode final : public VJoinNodeBase { void _probe_side_open_thread(RuntimeState* state, std::promise* status) override; private: + friend struct HashJoinProbeContext; + friend struct HashJoinBuildContext; + friend struct RuntimeFilterContext; + void _init_short_circuit_for_probe() override { _short_circuit_for_probe = (_short_circuit_for_null_in_probe_side && @@ -391,8 +685,6 @@ class HashJoinNode final : public VJoinNodeBase { void _hash_table_init(RuntimeState* state); void _process_hashtable_ctx_variants_init(RuntimeState* state); - static constexpr auto _MAX_BUILD_BLOCK_COUNT = 128; - void _prepare_probe_block(); static std::vector _convert_block_to_null(Block& block); @@ -417,6 +709,8 @@ class HashJoinNode final : public VJoinNodeBase { std::vector _runtime_filters; size_t _build_bf_cardinality = 0; std::atomic_bool _probe_open_finish = false; + + std::unique_ptr _probe_context; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/exec/scan/new_olap_scanner.cpp b/be/src/vec/exec/scan/new_olap_scanner.cpp index 862735ff359a4b1..30ec53362f36c0b 100644 --- a/be/src/vec/exec/scan/new_olap_scanner.cpp +++ b/be/src/vec/exec/scan/new_olap_scanner.cpp @@ -88,7 +88,7 @@ NewOlapScanner::NewOlapScanner(RuntimeState* state, NewOlapScanNode* parent, int _is_init = false; } -NewOlapScanner::NewOlapScanner(RuntimeState* state, pipeline::ScanLocalState* local_state, +NewOlapScanner::NewOlapScanner(RuntimeState* state, pipeline::ScanLocalStateBase* local_state, int64_t limit, bool aggregation, const TPaloScanRange& scan_range, const std::vector& key_ranges, RuntimeProfile* profile) @@ -101,7 +101,7 @@ NewOlapScanner::NewOlapScanner(RuntimeState* state, pipeline::ScanLocalState* lo _is_init = false; } -NewOlapScanner::NewOlapScanner(RuntimeState* state, pipeline::ScanLocalState* local_state, +NewOlapScanner::NewOlapScanner(RuntimeState* state, pipeline::ScanLocalStateBase* local_state, int64_t limit, bool aggregation, const TPaloScanRange& scan_range, const std::vector& key_ranges, const std::vector& rs_splits, RuntimeProfile* profile) @@ -472,10 +472,9 @@ Status NewOlapScanner::_init_return_columns() { : _tablet_schema->field_index(slot->col_name()); if (index < 0) { - std::stringstream ss; - ss << "field name is invalid. field=" << slot->col_name() - << ", field_name_to_index=" << _tablet_schema->get_all_field_names(); - return Status::InternalError(ss.str()); + return Status::InternalError( + "field name is invalid. field={}, field_name_to_index={}, col_unique_id={}", + slot->col_name(), _tablet_schema->get_all_field_names(), slot->col_unique_id()); } _return_columns.push_back(index); if (slot->is_nullable() && !_tablet_schema->column(index).is_nullable()) { diff --git a/be/src/vec/exec/scan/new_olap_scanner.h b/be/src/vec/exec/scan/new_olap_scanner.h index 49cbace77aed9f6..8b71c9e39cdce8b 100644 --- a/be/src/vec/exec/scan/new_olap_scanner.h +++ b/be/src/vec/exec/scan/new_olap_scanner.h @@ -61,11 +61,11 @@ class NewOlapScanner : public VScanner { const TPaloScanRange& scan_range, const std::vector& key_ranges, const std::vector& rs_splits, RuntimeProfile* profile); - NewOlapScanner(RuntimeState* state, pipeline::ScanLocalState* parent, int64_t limit, + NewOlapScanner(RuntimeState* state, pipeline::ScanLocalStateBase* parent, int64_t limit, bool aggregation, const TPaloScanRange& scan_range, const std::vector& key_ranges, RuntimeProfile* profile); - NewOlapScanner(RuntimeState* state, pipeline::ScanLocalState* parent, int64_t limit, + NewOlapScanner(RuntimeState* state, pipeline::ScanLocalStateBase* parent, int64_t limit, bool aggregation, const TPaloScanRange& scan_range, const std::vector& key_ranges, const std::vector& rs_splits, RuntimeProfile* profile); diff --git a/be/src/vec/exec/scan/pip_scanner_context.h b/be/src/vec/exec/scan/pip_scanner_context.h index 516076627670e18..39fc1845f6f5b9c 100644 --- a/be/src/vec/exec/scan/pip_scanner_context.h +++ b/be/src/vec/exec/scan/pip_scanner_context.h @@ -39,7 +39,7 @@ class PipScannerContext : public vectorized::ScannerContext { _col_distribute_ids(col_distribute_ids), _need_colocate_distribute(!_col_distribute_ids.empty()) {} - PipScannerContext(RuntimeState* state, ScanLocalState* local_state, + PipScannerContext(RuntimeState* state, ScanLocalStateBase* local_state, const TupleDescriptor* output_tuple_desc, const std::list& scanners, int64_t limit, int64_t max_bytes_in_blocks_queue, const std::vector& col_distribute_ids, diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index 8180dcd8f55b931..cbd4ae57be576e5 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -47,7 +47,7 @@ ScannerContext::ScannerContext(doris::RuntimeState* state_, doris::vectorized::V const doris::TupleDescriptor* output_tuple_desc, const std::list& scanners_, int64_t limit_, int64_t max_bytes_in_blocks_queue_, const int num_parallel_instances, - pipeline::ScanLocalState* local_state) + pipeline::ScanLocalStateBase* local_state) : _state(state_), _parent(parent), _local_state(local_state), @@ -188,6 +188,11 @@ bool ScannerContext::empty_in_queue(int id) { Status ScannerContext::get_block_from_queue(RuntimeState* state, vectorized::BlockUPtr* block, bool* eos, int id, bool wait) { std::unique_lock l(_transfer_lock); + // debug case failure, to be removed + if (state->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(state->query_id()) << " " + << _parent->get_name() << ": ScannerContext::get_block_from_queue"; + } // Normally, the scanner scheduler will schedule ctx. // But when the amount of data in the blocks queue exceeds the upper limit, // the scheduler will stop scheduling. @@ -204,6 +209,7 @@ Status ScannerContext::get_block_from_queue(RuntimeState* state, vectorized::Blo } // Wait for block from queue if (wait) { + // scanner batch wait time SCOPED_TIMER(_scanner_wait_batch_timer); while (!(!_blocks_queue.empty() || _is_finished || !status().ok() || state->is_cancelled())) { @@ -419,7 +425,8 @@ taskgroup::TaskGroup* ScannerContext::get_task_group() const { return _state->get_query_ctx()->get_task_group(); } -template void ScannerContext::clear_and_join(pipeline::ScanLocalState* parent, RuntimeState* state); +template void ScannerContext::clear_and_join(pipeline::ScanLocalStateBase* parent, + RuntimeState* state); template void ScannerContext::clear_and_join(VScanNode* parent, RuntimeState* state); } // namespace doris::vectorized diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index 6ce9d01286a74e9..dd0439e2780faff 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -42,7 +42,7 @@ class RuntimeState; class TupleDescriptor; namespace pipeline { -class ScanLocalState; +class ScanLocalStateBase; } // namespace pipeline namespace taskgroup { @@ -71,7 +71,7 @@ class ScannerContext { const TupleDescriptor* output_tuple_desc, const std::list& scanners_, int64_t limit_, int64_t max_bytes_in_blocks_queue_, const int num_parallel_instances = 0, - pipeline::ScanLocalState* local_state = nullptr); + pipeline::ScanLocalStateBase* local_state = nullptr); virtual ~ScannerContext() = default; virtual Status init(); @@ -176,7 +176,7 @@ class ScannerContext { RuntimeState* _state; VScanNode* _parent; - pipeline::ScanLocalState* _local_state; + pipeline::ScanLocalStateBase* _local_state; // the comment of same fields in VScanNode const TupleDescriptor* _output_tuple_desc; diff --git a/be/src/vec/exec/scan/scanner_scheduler.cpp b/be/src/vec/exec/scan/scanner_scheduler.cpp index 291f5783e8ea97f..abe6d818af9992a 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.cpp +++ b/be/src/vec/exec/scan/scanner_scheduler.cpp @@ -219,6 +219,12 @@ void ScannerScheduler::_schedule_scanners(ScannerContext* ctx) { this_run.erase(iter++); } else { ctx->set_status_on_error(s); + // debug case failure, to be removed + if (ctx->state()->enable_profile()) { + LOG(WARNING) + << "debug case failure " << print_id(ctx->state()->query_id()) + << " " << ctx->parent()->get_name() << ": submit_func error: " << s; + } break; } } @@ -254,6 +260,11 @@ void ScannerScheduler::_schedule_scanners(ScannerContext* ctx) { } else { ctx->set_status_on_error( Status::InternalError("failed to submit scanner to scanner pool")); + // debug case failure, to be removed + if (ctx->state()->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(ctx->state()->query_id()) + << " " << ctx->parent()->get_name() << ": submit_func error2"; + } break; } } @@ -262,6 +273,11 @@ void ScannerScheduler::_schedule_scanners(ScannerContext* ctx) { #if !defined(USE_BTHREAD_SCANNER) submit_to_thread_pool(); #else + // debug case failure, to be removed + if (ctx->state()->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(ctx->state()->query_id()) << " " + << ctx->parent()->get_name() << ": USE_BTHREAD_SCANNER"; + } // Only OlapScanner uses bthread scanner // Todo: Make other scanners support bthread scanner if (dynamic_cast(*iter) == nullptr) { @@ -302,6 +318,11 @@ void ScannerScheduler::_schedule_scanners(ScannerContext* ctx) { void ScannerScheduler::_scanner_scan(ScannerScheduler* scheduler, ScannerContext* ctx, VScannerSPtr scanner) { + // debug case failure, to be removed + if (ctx->state()->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(ctx->state()->query_id()) << " " + << ctx->parent()->get_name() << ": ScannerScheduler::_scanner_scan"; + } SCOPED_ATTACH_TASK(scanner->runtime_state()); #if !defined(USE_BTHREAD_SCANNER) Thread::set_self_name("_scanner_scan"); @@ -321,6 +342,12 @@ void ScannerScheduler::_scanner_scan(ScannerScheduler* scheduler, ScannerContext if (!status.ok()) { ctx->set_status_on_error(status); eos = true; + // debug case failure, to be removed + if (ctx->state()->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(ctx->state()->query_id()) << " " + << ctx->parent()->get_name() + << ": ScannerScheduler::_scanner_scan scanner->init eos"; + } } } if (!eos && !scanner->is_open()) { @@ -328,6 +355,12 @@ void ScannerScheduler::_scanner_scan(ScannerScheduler* scheduler, ScannerContext if (!status.ok()) { ctx->set_status_on_error(status); eos = true; + // debug case failure, to be removed + if (ctx->state()->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(ctx->state()->query_id()) << " " + << ctx->parent()->get_name() + << ": ScannerScheduler::_scanner_scan scanner->open eos"; + } } scanner->set_opened(); } @@ -363,6 +396,12 @@ void ScannerScheduler::_scanner_scan(ScannerScheduler* scheduler, ScannerContext // No need to set status on error here. // Because done() maybe caused by "should_stop" should_stop = true; + // debug case failure, to be removed + if (ctx->state()->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(ctx->state()->query_id()) << " " + << ctx->parent()->get_name() + << ": ScannerScheduler::_scanner_scan ctx->done"; + } break; } diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp index 612dc32b2248966..b787c16224eea0f 100644 --- a/be/src/vec/exec/scan/vscan_node.cpp +++ b/be/src/vec/exec/scan/vscan_node.cpp @@ -226,6 +226,11 @@ Status VScanNode::alloc_resource(RuntimeState* state) { } Status VScanNode::get_next(RuntimeState* state, vectorized::Block* block, bool* eos) { + // debug case failure, to be removed + if (state->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(state->query_id()) << " " << get_name() + << ": VScanNode::get_next"; + } SCOPED_TIMER(_get_next_timer); SCOPED_TIMER(_runtime_profile->total_time_counter()); // in inverted index apply logic, in order to optimize query performance, @@ -242,6 +247,11 @@ Status VScanNode::get_next(RuntimeState* state, vectorized::Block* block, bool* }}; if (state->is_cancelled()) { + // debug case failure, to be removed + if (state->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(state->query_id()) << " " + << get_name() << ": VScanNode::get_next canceled"; + } // ISSUE: https://github.com/apache/doris/issues/16360 // _scanner_ctx may be null here, see: `VScanNode::alloc_resource` (_eos == null) if (_scanner_ctx) { @@ -254,6 +264,11 @@ Status VScanNode::get_next(RuntimeState* state, vectorized::Block* block, bool* if (_eos) { *eos = true; + // debug case failure, to be removed + if (state->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(state->query_id()) << " " + << get_name() << ": VScanNode::get_next eos"; + } return Status::OK(); } diff --git a/be/src/vec/exec/scan/vscanner.cpp b/be/src/vec/exec/scan/vscanner.cpp index 64fb402ff8517f2..2008f899214da13 100644 --- a/be/src/vec/exec/scan/vscanner.cpp +++ b/be/src/vec/exec/scan/vscanner.cpp @@ -39,7 +39,7 @@ VScanner::VScanner(RuntimeState* state, VScanNode* parent, int64_t limit, Runtim _total_rf_num = _parent->runtime_filter_num(); } -VScanner::VScanner(RuntimeState* state, pipeline::ScanLocalState* local_state, int64_t limit, +VScanner::VScanner(RuntimeState* state, pipeline::ScanLocalStateBase* local_state, int64_t limit, RuntimeProfile* profile) : _state(state), _parent(nullptr), @@ -62,8 +62,14 @@ Status VScanner::prepare(RuntimeState* state, const VExprContextSPtrs& conjuncts } Status VScanner::get_block(RuntimeState* state, Block* block, bool* eof) { + // debug case failure, to be removed + if (state->enable_profile()) { + LOG(WARNING) << "debug case failure " << print_id(state->query_id()) << " " + << _parent->get_name() << ": VScanner::get_block"; + } // only empty block should be here DCHECK(block->rows() == 0); + // scanner running time SCOPED_RAW_TIMER(&_per_scanner_timer); int64_t rows_read_threshold = _num_rows_read + config::doris_scanner_row_num; if (!block->mem_reuse()) { @@ -85,6 +91,7 @@ Status VScanner::get_block(RuntimeState* state, Block* block, bool* eof) { block->clear_same_bit(); // 1. Get input block from scanner { + // get block time auto timer = _parent ? _parent->_scan_timer : _local_state->_scan_timer; SCOPED_TIMER(timer); RETURN_IF_ERROR(_get_block_impl(state, block, eof)); diff --git a/be/src/vec/exec/scan/vscanner.h b/be/src/vec/exec/scan/vscanner.h index ee601cf684ce887..5359204758fc635 100644 --- a/be/src/vec/exec/scan/vscanner.h +++ b/be/src/vec/exec/scan/vscanner.h @@ -38,7 +38,7 @@ class VExprContext; } // namespace vectorized namespace pipeline { -class ScanLocalState; +class ScanLocalStateBase; } // namespace pipeline } // namespace doris @@ -57,7 +57,7 @@ struct ScannerCounter { class VScanner { public: VScanner(RuntimeState* state, VScanNode* parent, int64_t limit, RuntimeProfile* profile); - VScanner(RuntimeState* state, pipeline::ScanLocalState* local_state, int64_t limit, + VScanner(RuntimeState* state, pipeline::ScanLocalStateBase* local_state, int64_t limit, RuntimeProfile* profile); virtual ~VScanner() = default; @@ -162,7 +162,7 @@ class VScanner { RuntimeState* _state; VScanNode* _parent; - pipeline::ScanLocalState* _local_state; + pipeline::ScanLocalStateBase* _local_state; // Set if scan node has sort limit info int64_t _limit = -1; diff --git a/be/src/vec/exec/vtable_function_node.h b/be/src/vec/exec/vtable_function_node.h index 040ca3f7afef160..db88147cf44bded 100644 --- a/be/src/vec/exec/vtable_function_node.h +++ b/be/src/vec/exec/vtable_function_node.h @@ -69,14 +69,14 @@ class VTableFunctionNode final : public ExecNode { ExecNode::release_resource(state); } - Status push(RuntimeState*, Block* input_block, bool eos) override { + Status push(RuntimeState* state, Block* input_block, bool eos) override { _child_eos = eos; if (input_block->rows() == 0) { return Status::OK(); } for (TableFunction* fn : _fns) { - RETURN_IF_ERROR(fn->process_init(input_block)); + RETURN_IF_ERROR(fn->process_init(input_block, state)); } RETURN_IF_ERROR(_process_next_child_row()); return Status::OK(); diff --git a/be/src/vec/exec/vunion_node.cpp b/be/src/vec/exec/vunion_node.cpp index e9caa494fd437d8..90cbe5bb7aeed8b 100644 --- a/be/src/vec/exec/vunion_node.cpp +++ b/be/src/vec/exec/vunion_node.cpp @@ -106,6 +106,12 @@ Status VUnionNode::open(RuntimeState* state) { Status VUnionNode::alloc_resource(RuntimeState* state) { SCOPED_TIMER(_runtime_profile->total_time_counter()); + + std::unique_lock l(_resource_lock); + if (_resource_allocated) { + return Status::OK(); + } + // open const expr lists. for (const auto& exprs : _const_expr_lists) { RETURN_IF_ERROR(VExpr::open(exprs, state)); @@ -114,7 +120,9 @@ Status VUnionNode::alloc_resource(RuntimeState* state) { for (const auto& exprs : _child_expr_lists) { RETURN_IF_ERROR(VExpr::open(exprs, state)); } - return ExecNode::alloc_resource(state); + RETURN_IF_ERROR(ExecNode::alloc_resource(state)); + _resource_allocated = true; + return Status::OK(); } Status VUnionNode::get_next_pass_through(RuntimeState* state, Block* block) { diff --git a/be/src/vec/exec/vunion_node.h b/be/src/vec/exec/vunion_node.h index c25bb07102ba572..ac63f9ca6323c0b 100644 --- a/be/src/vec/exec/vunion_node.h +++ b/be/src/vec/exec/vunion_node.h @@ -91,6 +91,9 @@ class VUnionNode final : public ExecNode { /// to -1 if no child needs to be closed. int _to_close_child_idx; + std::mutex _resource_lock; + bool _resource_allocated {false}; + // Time spent to evaluates exprs and materializes the results RuntimeProfile::Counter* _materialize_exprs_evaluate_timer = nullptr; /// GetNext() for the passthrough case. We pass 'block' directly into the GetNext() diff --git a/be/src/vec/exprs/table_function/table_function.h b/be/src/vec/exprs/table_function/table_function.h index 4b31a681b11a731..7ba6379a204e8ae 100644 --- a/be/src/vec/exprs/table_function/table_function.h +++ b/be/src/vec/exprs/table_function/table_function.h @@ -36,10 +36,12 @@ class TableFunction { virtual Status open() { return Status::OK(); } - virtual Status process_init(Block* block) = 0; + virtual Status process_init(Block* block, RuntimeState* state) = 0; virtual Status process_row(size_t row_idx) { - _cur_size = 0; + if (!_is_const) { + _cur_size = 0; + } return reset(); } diff --git a/be/src/vec/exprs/table_function/vexplode.cpp b/be/src/vec/exprs/table_function/vexplode.cpp index 1340ff74bb7132e..6f13a710e27a2a1 100644 --- a/be/src/vec/exprs/table_function/vexplode.cpp +++ b/be/src/vec/exprs/table_function/vexplode.cpp @@ -36,7 +36,7 @@ VExplodeTableFunction::VExplodeTableFunction() { _fn_name = "vexplode"; } -Status VExplodeTableFunction::process_init(Block* block) { +Status VExplodeTableFunction::process_init(Block* block, RuntimeState* state) { CHECK(_expr_context->root()->children().size() == 1) << "VExplodeTableFunction only support 1 child but has " << _expr_context->root()->children().size(); diff --git a/be/src/vec/exprs/table_function/vexplode.h b/be/src/vec/exprs/table_function/vexplode.h index 26771bd2b272819..2da97d11a40c1e1 100644 --- a/be/src/vec/exprs/table_function/vexplode.h +++ b/be/src/vec/exprs/table_function/vexplode.h @@ -40,7 +40,7 @@ class VExplodeTableFunction : public TableFunction { ~VExplodeTableFunction() override = default; - Status process_init(Block* block) override; + Status process_init(Block* block, RuntimeState* state) override; Status process_row(size_t row_idx) override; Status process_close() override; void get_value(MutableColumnPtr& column) override; diff --git a/be/src/vec/exprs/table_function/vexplode_bitmap.cpp b/be/src/vec/exprs/table_function/vexplode_bitmap.cpp index 152566d00bb28cf..9ce77714549e2ae 100644 --- a/be/src/vec/exprs/table_function/vexplode_bitmap.cpp +++ b/be/src/vec/exprs/table_function/vexplode_bitmap.cpp @@ -40,7 +40,7 @@ VExplodeBitmapTableFunction::VExplodeBitmapTableFunction() { _fn_name = "vexplode_bitmap"; } -Status VExplodeBitmapTableFunction::process_init(Block* block) { +Status VExplodeBitmapTableFunction::process_init(Block* block, RuntimeState* state) { CHECK(_expr_context->root()->children().size() == 1) << "VExplodeNumbersTableFunction must be have 1 children but have " << _expr_context->root()->children().size(); diff --git a/be/src/vec/exprs/table_function/vexplode_bitmap.h b/be/src/vec/exprs/table_function/vexplode_bitmap.h index 0dfde605de3babf..18cdc6bb22c79ea 100644 --- a/be/src/vec/exprs/table_function/vexplode_bitmap.h +++ b/be/src/vec/exprs/table_function/vexplode_bitmap.h @@ -45,7 +45,7 @@ class VExplodeBitmapTableFunction final : public TableFunction { void get_value(MutableColumnPtr& column) override; Status forward(int step = 1) override; - Status process_init(Block* block) override; + Status process_init(Block* block, RuntimeState* state) override; Status process_row(size_t row_idx) override; Status process_close() override; diff --git a/be/src/vec/exprs/table_function/vexplode_json_array.cpp b/be/src/vec/exprs/table_function/vexplode_json_array.cpp index 840923d0181dafb..7c8c48733f4ab49 100644 --- a/be/src/vec/exprs/table_function/vexplode_json_array.cpp +++ b/be/src/vec/exprs/table_function/vexplode_json_array.cpp @@ -162,7 +162,7 @@ VExplodeJsonArrayTableFunction::VExplodeJsonArrayTableFunction(ExplodeJsonArrayT _fn_name = "vexplode_json_array"; } -Status VExplodeJsonArrayTableFunction::process_init(Block* block) { +Status VExplodeJsonArrayTableFunction::process_init(Block* block, RuntimeState* state) { CHECK(_expr_context->root()->children().size() == 1) << _expr_context->root()->children().size(); diff --git a/be/src/vec/exprs/table_function/vexplode_json_array.h b/be/src/vec/exprs/table_function/vexplode_json_array.h index db1784e5ad3405f..6b5a0f76513700d 100644 --- a/be/src/vec/exprs/table_function/vexplode_json_array.h +++ b/be/src/vec/exprs/table_function/vexplode_json_array.h @@ -111,7 +111,7 @@ class VExplodeJsonArrayTableFunction final : public TableFunction { VExplodeJsonArrayTableFunction(ExplodeJsonArrayType type); ~VExplodeJsonArrayTableFunction() override = default; - Status process_init(Block* block) override; + Status process_init(Block* block, RuntimeState* state) override; Status process_row(size_t row_idx) override; Status process_close() override; void get_value(MutableColumnPtr& column) override; diff --git a/be/src/vec/exprs/table_function/vexplode_numbers.cpp b/be/src/vec/exprs/table_function/vexplode_numbers.cpp index fe450c809605715..8149d4d9583f63c 100644 --- a/be/src/vec/exprs/table_function/vexplode_numbers.cpp +++ b/be/src/vec/exprs/table_function/vexplode_numbers.cpp @@ -23,6 +23,7 @@ #include #include "common/status.h" +#include "runtime/runtime_state.h" #include "vec/columns/column.h" #include "vec/columns/column_const.h" #include "vec/columns/column_nullable.h" @@ -40,7 +41,7 @@ VExplodeNumbersTableFunction::VExplodeNumbersTableFunction() { _fn_name = "vexplode_numbers"; } -Status VExplodeNumbersTableFunction::process_init(Block* block) { +Status VExplodeNumbersTableFunction::process_init(Block* block, RuntimeState* state) { CHECK(_expr_context->root()->children().size() == 1) << "VExplodeSplitTableFunction must be have 1 children but have " << _expr_context->root()->children().size(); @@ -61,9 +62,10 @@ Status VExplodeNumbersTableFunction::process_init(Block* block) { } else { _cur_size = column_nested->get_int(0); } - - if (_cur_size && _cur_size <= block->rows()) { // avoid elements_column too big or empty - _is_const = true; // use const optimize + ((ColumnInt32*)_elements_column.get())->clear(); + if (_cur_size && + _cur_size <= state->batch_size()) { // avoid elements_column too big or empty + _is_const = true; // use const optimize for (int i = 0; i < _cur_size; i++) { ((ColumnInt32*)_elements_column.get())->insert_value(i); } diff --git a/be/src/vec/exprs/table_function/vexplode_numbers.h b/be/src/vec/exprs/table_function/vexplode_numbers.h index f4f86e8d4d375f0..7bdde9278bd258a 100644 --- a/be/src/vec/exprs/table_function/vexplode_numbers.h +++ b/be/src/vec/exprs/table_function/vexplode_numbers.h @@ -43,7 +43,7 @@ class VExplodeNumbersTableFunction : public TableFunction { VExplodeNumbersTableFunction(); ~VExplodeNumbersTableFunction() override = default; - Status process_init(Block* block) override; + Status process_init(Block* block, RuntimeState* state) override; Status process_row(size_t row_idx) override; Status process_close() override; void get_value(MutableColumnPtr& column) override; @@ -53,13 +53,13 @@ class VExplodeNumbersTableFunction : public TableFunction { if (_is_nullable) { static_cast( static_cast(column.get())->get_nested_column_ptr().get()) - ->insert_many_from(*_elements_column, _cur_offset, max_step); + ->insert_range_from(*_elements_column, _cur_offset, max_step); static_cast( static_cast(column.get())->get_null_map_column_ptr().get()) ->insert_many_defaults(max_step); } else { static_cast(column.get()) - ->insert_many_from(*_elements_column, _cur_offset, max_step); + ->insert_range_from(*_elements_column, _cur_offset, max_step); } forward(max_step); diff --git a/be/src/vec/exprs/table_function/vexplode_split.cpp b/be/src/vec/exprs/table_function/vexplode_split.cpp index c3ae83f9f35a25b..c2a48f456adf409 100644 --- a/be/src/vec/exprs/table_function/vexplode_split.cpp +++ b/be/src/vec/exprs/table_function/vexplode_split.cpp @@ -43,7 +43,7 @@ Status VExplodeSplitTableFunction::open() { return Status::OK(); } -Status VExplodeSplitTableFunction::process_init(Block* block) { +Status VExplodeSplitTableFunction::process_init(Block* block, RuntimeState* state) { CHECK(_expr_context->root()->children().size() == 2) << "VExplodeSplitTableFunction must be have 2 children but have " << _expr_context->root()->children().size(); diff --git a/be/src/vec/exprs/table_function/vexplode_split.h b/be/src/vec/exprs/table_function/vexplode_split.h index 1155090bb17a1c1..629e2259b65bace 100644 --- a/be/src/vec/exprs/table_function/vexplode_split.h +++ b/be/src/vec/exprs/table_function/vexplode_split.h @@ -45,7 +45,7 @@ class VExplodeSplitTableFunction final : public TableFunction { ~VExplodeSplitTableFunction() override = default; Status open() override; - Status process_init(Block* block) override; + Status process_init(Block* block, RuntimeState* state) override; Status process_row(size_t row_idx) override; Status process_close() override; void get_value(MutableColumnPtr& column) override; diff --git a/be/src/vec/functions/function_binary_arithmetic.h b/be/src/vec/functions/function_binary_arithmetic.h index 0b92898302b802f..1725ef30a20c13e 100644 --- a/be/src/vec/functions/function_binary_arithmetic.h +++ b/be/src/vec/functions/function_binary_arithmetic.h @@ -459,7 +459,7 @@ struct DecimalBinaryOperation { NativeResultType res; // TODO handle overflow gracefully if (Op::template apply(a, b, res)) { - res = max_decimal_value(); + res = type_limit::max(); } return res; } else { @@ -499,7 +499,7 @@ struct DecimalBinaryOperation { // TODO handle overflow gracefully if (overflow) { LOG(WARNING) << "Decimal math overflow"; - res = max_decimal_value(); + res = type_limit::max(); } } else { res = apply(a, b); diff --git a/be/src/vec/functions/function_java_udf.cpp b/be/src/vec/functions/function_java_udf.cpp index 7c50e74117716b2..fa9c8517c8e74b3 100644 --- a/be/src/vec/functions/function_java_udf.cpp +++ b/be/src/vec/functions/function_java_udf.cpp @@ -121,9 +121,9 @@ Status JavaFunctionCall::open(FunctionContext* context, FunctionContext::Functio return Status::OK(); } -Status JavaFunctionCall::execute(FunctionContext* context, Block& block, - const ColumnNumbers& arguments, size_t result, size_t num_rows, - bool dry_run) { +Status JavaFunctionCall::execute_impl(FunctionContext* context, Block& block, + const ColumnNumbers& arguments, size_t result, + size_t num_rows) const { JNIEnv* env = nullptr; RETURN_IF_ERROR(JniUtil::GetJNIEnv(&env)); JniContext* jni_ctx = reinterpret_cast( diff --git a/be/src/vec/functions/function_java_udf.h b/be/src/vec/functions/function_java_udf.h index ddbe300e89c4544..875e3426df1faf3 100644 --- a/be/src/vec/functions/function_java_udf.h +++ b/be/src/vec/functions/function_java_udf.h @@ -22,6 +22,7 @@ #include #include +#include #include #include #include @@ -37,10 +38,35 @@ #include "vec/core/types.h" #include "vec/data_types/data_type.h" #include "vec/functions/function.h" - namespace doris { namespace vectorized { + +class JavaUdfPreparedFunction : public PreparedFunctionImpl { +public: + using execute_call_back = std::function; + + explicit JavaUdfPreparedFunction(const execute_call_back& func, const std::string& name) + : callback_function(func), name(name) {} + + String get_name() const override { return name; } + +protected: + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) override { + return callback_function(context, block, arguments, result, input_rows_count); + } + + bool use_default_implementation_for_nulls() const override { return false; } + bool use_default_implementation_for_low_cardinality_columns() const override { return false; } + +private: + execute_call_back callback_function; + std::string name; +}; + class JavaFunctionCall : public IFunctionBase { public: JavaFunctionCall(const TFunction& fn, const DataTypes& argument_types, @@ -63,13 +89,17 @@ class JavaFunctionCall : public IFunctionBase { PreparedFunctionPtr prepare(FunctionContext* context, const Block& sample_block, const ColumnNumbers& arguments, size_t result) const override { - return nullptr; + return std::make_shared( + std::bind(&JavaFunctionCall::execute_impl, this, std::placeholders::_1, + std::placeholders::_2, std::placeholders::_3, + std::placeholders::_4, std::placeholders::_5), + fn_.name.function_name); } Status open(FunctionContext* context, FunctionContext::FunctionStateScope scope) override; - Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments, - size_t result, size_t input_rows_count, bool dry_run = false) override; + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) const; Status close(FunctionContext* context, FunctionContext::FunctionStateScope scope) override; diff --git a/be/src/vec/functions/function_regexp.cpp b/be/src/vec/functions/function_regexp.cpp index 50f845acc3c341c..31ad61cc187ad37 100644 --- a/be/src/vec/functions/function_regexp.cpp +++ b/be/src/vec/functions/function_regexp.cpp @@ -350,6 +350,11 @@ struct RegexpExtractAllImpl { StringOP::push_empty_string(index_now, result_data, result_offset); break; } + if (matches[0].empty()) { + StringOP::push_empty_string(index_now, result_data, result_offset); + pos += 1; + continue; + } res_matches.push_back(matches[1]); auto offset = std::string(str_pos, str_size).find(std::string(matches[0].as_string())); pos += offset + matches[0].size(); diff --git a/be/src/vec/functions/like.cpp b/be/src/vec/functions/like.cpp index 1a83dc8a87804ce..8851b777fee6e62 100644 --- a/be/src/vec/functions/like.cpp +++ b/be/src/vec/functions/like.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "vec/functions/like.h" +#include "like.h" #include #include @@ -148,68 +148,6 @@ Status FunctionLikeBase::constant_substring_fn(LikeSearchState* state, const Col return Status::OK(); } -Status FunctionLikeBase::constant_allpass_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, - const uint16_t* sel, size_t sz) { - for (size_t i = 0; i < sz; i++) { - result[i] = 1; - } - return Status::OK(); -} - -Status FunctionLikeBase::constant_starts_with_fn_predicate( - LikeSearchState* state, const PredicateColumnType& val, - const StringRef& pattern, ColumnUInt8::Container& result, const uint16_t* sel, size_t sz) { - auto data_ptr = reinterpret_cast(val.get_data().data()); - for (size_t i = 0; i < sz; i++) { - result[i] = (data_ptr[sel[i]].size >= state->search_string_sv.size) && - (state->search_string_sv == - data_ptr[sel[i]].substring(0, state->search_string_sv.size)); - } - return Status::OK(); -} - -Status FunctionLikeBase::constant_ends_with_fn_predicate( - LikeSearchState* state, const PredicateColumnType& val, - const StringRef& pattern, ColumnUInt8::Container& result, const uint16_t* sel, size_t sz) { - auto data_ptr = reinterpret_cast(val.get_data().data()); - for (size_t i = 0; i < sz; i++) { - result[i] = - (data_ptr[sel[i]].size >= state->search_string_sv.size) && - (state->search_string_sv == - data_ptr[sel[i]].substring(data_ptr[sel[i]].size - state->search_string_sv.size, - state->search_string_sv.size)); - } - return Status::OK(); -} - -Status FunctionLikeBase::constant_equals_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, - const uint16_t* sel, size_t sz) { - auto data_ptr = reinterpret_cast(val.get_data().data()); - for (size_t i = 0; i < sz; i++) { - result[i] = (data_ptr[sel[i]] == state->search_string_sv); - } - return Status::OK(); -} - -Status FunctionLikeBase::constant_substring_fn_predicate( - LikeSearchState* state, const PredicateColumnType& val, - const StringRef& pattern, ColumnUInt8::Container& result, const uint16_t* sel, size_t sz) { - auto data_ptr = reinterpret_cast(val.get_data().data()); - for (size_t i = 0; i < sz; i++) { - if (state->search_string_sv.size == 0) { - result[i] = true; - } - result[i] = state->substring_pattern.search(data_ptr[sel[i]]) != -1; - } - return Status::OK(); -} - Status FunctionLikeBase::constant_allpass_fn_scalar(LikeSearchState* state, const StringRef& val, const StringRef& pattern, unsigned char* result) { @@ -351,75 +289,6 @@ Status FunctionLikeBase::regexp_fn(LikeSearchState* state, const ColumnString& v return Status::OK(); } -Status FunctionLikeBase::constant_regex_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, - const uint16_t* sel, size_t sz) { - auto data_ptr = reinterpret_cast(val.get_data().data()); - - if (state->hs_database) { // use hyperscan - for (size_t i = 0; i < sz; i++) { - auto ret = hs_scan(state->hs_database.get(), data_ptr[sel[i]].data, - data_ptr[sel[i]].size, 0, state->hs_scratch.get(), - doris::vectorized::LikeSearchState::hs_match_handler, - (void*)(result.data() + i)); - if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) { - return Status::RuntimeError(fmt::format("hyperscan error: {}", ret)); - } - } - } else { // fallback to re2 - for (size_t i = 0; i < sz; i++) { - *(result.data() + i) = RE2::PartialMatch( - re2::StringPiece(data_ptr[sel[i]].data, data_ptr[sel[i]].size), - *state->regex.get()); - } - } - - return Status::OK(); -} - -Status FunctionLikeBase::regexp_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, const uint16_t* sel, - size_t sz) { - std::string re_pattern(pattern.data, pattern.size); - - hs_database_t* database = nullptr; - hs_scratch_t* scratch = nullptr; - if (hs_prepare(nullptr, re_pattern.c_str(), &database, &scratch).ok()) { // use hyperscan - auto data_ptr = reinterpret_cast(val.get_data().data()); - for (size_t i = 0; i < sz; i++) { - auto ret = hs_scan(database, data_ptr[sel[i]].data, data_ptr[sel[i]].size, 0, scratch, - doris::vectorized::LikeSearchState::hs_match_handler, - (void*)(result.data() + i)); - if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) { - return Status::RuntimeError(fmt::format("hyperscan error: {}", ret)); - } - } - - hs_free_scratch(scratch); - hs_free_database(database); - } else { // fallback to re2 - RE2::Options opts; - opts.set_never_nl(false); - opts.set_dot_nl(true); - re2::RE2 re(re_pattern, opts); - if (re.ok()) { - auto data_ptr = reinterpret_cast(val.get_data().data()); - for (size_t i = 0; i < sz; i++) { - *(result.data() + i) = RE2::PartialMatch( - re2::StringPiece(data_ptr[sel[i]].data, data_ptr[sel[i]].size), re); - } - } else { - return Status::RuntimeError("Invalid pattern: {}", pattern.debug_string()); - } - } - - return Status::OK(); -} - // hyperscan compile expression to database and allocate scratch space Status FunctionLikeBase::hs_prepare(FunctionContext* context, const char* expression, hs_database_t** database, hs_scratch_t** scratch) { @@ -559,17 +428,6 @@ Status FunctionLike::like_fn(LikeSearchState* state, const ColumnString& val, return regexp_fn(state, val, {re_pattern.c_str(), re_pattern.size()}, result); } -Status FunctionLike::like_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, ColumnUInt8::Container& result, - uint16_t* sel, size_t sz) { - std::string re_pattern; - convert_like_pattern(state, std::string(pattern.data, pattern.size), &re_pattern); - - return regexp_fn_predicate(state, val, {re_pattern.c_str(), re_pattern.size()}, result, sel, - sz); -} - Status FunctionLike::like_fn_scalar(LikeSearchState* state, const StringRef& val, const StringRef& pattern, unsigned char* result) { std::string re_pattern; @@ -680,7 +538,6 @@ Status FunctionLike::open(FunctionContext* context, FunctionContext::FunctionSta std::shared_ptr state = std::make_shared(); context->set_function_state(scope, state); state->function = like_fn; - state->predicate_like_function = like_fn_predicate; state->scalar_function = like_fn_scalar; if (context->is_col_constant(1)) { const auto pattern_col = context->get_constant_col(1)->column_ptr; @@ -693,7 +550,6 @@ Status FunctionLike::open(FunctionContext* context, FunctionContext::FunctionSta if (!pattern_str.empty() && RE2::FullMatch(pattern_str, LIKE_ALLPASS_RE)) { state->search_state.set_search_string(""); state->function = constant_allpass_fn; - state->predicate_like_function = constant_allpass_fn_predicate; state->scalar_function = constant_allpass_fn_scalar; } else if (pattern_str.empty() || RE2::FullMatch(pattern_str, LIKE_EQUALS_RE, &search_string)) { @@ -709,7 +565,6 @@ Status FunctionLike::open(FunctionContext* context, FunctionContext::FunctionSta } state->search_state.set_search_string(search_string); state->function = constant_equals_fn; - state->predicate_like_function = constant_equals_fn_predicate; state->scalar_function = constant_equals_fn_scalar; } else if (RE2::FullMatch(pattern_str, LIKE_STARTS_WITH_RE, &search_string)) { if (VLOG_DEBUG_IS_ON) { @@ -724,7 +579,6 @@ Status FunctionLike::open(FunctionContext* context, FunctionContext::FunctionSta } state->search_state.set_search_string(search_string); state->function = constant_starts_with_fn; - state->predicate_like_function = constant_starts_with_fn_predicate; state->scalar_function = constant_starts_with_fn_scalar; } else if (RE2::FullMatch(pattern_str, LIKE_ENDS_WITH_RE, &search_string)) { if (VLOG_DEBUG_IS_ON) { @@ -739,7 +593,6 @@ Status FunctionLike::open(FunctionContext* context, FunctionContext::FunctionSta } state->search_state.set_search_string(search_string); state->function = constant_ends_with_fn; - state->predicate_like_function = constant_ends_with_fn_predicate; state->scalar_function = constant_ends_with_fn_scalar; } else if (RE2::FullMatch(pattern_str, LIKE_SUBSTRING_RE, &search_string)) { if (VLOG_DEBUG_IS_ON) { @@ -754,7 +607,6 @@ Status FunctionLike::open(FunctionContext* context, FunctionContext::FunctionSta } state->search_state.set_search_string(search_string); state->function = constant_substring_fn; - state->predicate_like_function = constant_substring_fn_predicate; state->scalar_function = constant_substring_fn_scalar; } else { std::string re_pattern; @@ -787,7 +639,6 @@ Status FunctionLike::open(FunctionContext* context, FunctionContext::FunctionSta } state->function = constant_regex_fn; - state->predicate_like_function = constant_regex_fn_predicate; state->scalar_function = constant_regex_fn_scalar; } } @@ -801,7 +652,6 @@ Status FunctionRegexp::open(FunctionContext* context, FunctionContext::FunctionS std::shared_ptr state = std::make_shared(); context->set_function_state(scope, state); state->function = regexp_fn; - state->predicate_like_function = regexp_fn_predicate; state->scalar_function = regexp_fn_scalar; if (context->is_col_constant(1)) { const auto pattern_col = context->get_constant_col(1)->column_ptr; @@ -812,27 +662,22 @@ Status FunctionRegexp::open(FunctionContext* context, FunctionContext::FunctionS if (RE2::FullMatch(pattern_str, ALLPASS_RE)) { state->search_state.set_search_string(""); state->function = constant_allpass_fn; - state->predicate_like_function = constant_allpass_fn_predicate; state->scalar_function = constant_allpass_fn_scalar; } else if (RE2::FullMatch(pattern_str, EQUALS_RE, &search_string)) { state->search_state.set_search_string(search_string); state->function = constant_equals_fn; - state->predicate_like_function = constant_equals_fn_predicate; state->scalar_function = constant_equals_fn_scalar; } else if (RE2::FullMatch(pattern_str, STARTS_WITH_RE, &search_string)) { state->search_state.set_search_string(search_string); state->function = constant_starts_with_fn; - state->predicate_like_function = constant_starts_with_fn_predicate; state->scalar_function = constant_starts_with_fn_scalar; } else if (RE2::FullMatch(pattern_str, ENDS_WITH_RE, &search_string)) { state->search_state.set_search_string(search_string); state->function = constant_ends_with_fn; - state->predicate_like_function = constant_ends_with_fn_predicate; state->scalar_function = constant_ends_with_fn_scalar; } else if (RE2::FullMatch(pattern_str, SUBSTRING_RE, &search_string)) { state->search_state.set_search_string(search_string); state->function = constant_substring_fn; - state->predicate_like_function = constant_substring_fn_predicate; state->scalar_function = constant_substring_fn_scalar; } else { hs_database_t* database = nullptr; @@ -855,7 +700,6 @@ Status FunctionRegexp::open(FunctionContext* context, FunctionContext::FunctionS } } state->function = constant_regex_fn; - state->predicate_like_function = constant_regex_fn_predicate; state->scalar_function = constant_regex_fn_scalar; } } diff --git a/be/src/vec/functions/like.h b/be/src/vec/functions/like.h index 07309be5b74e734..3726518a9b93604 100644 --- a/be/src/vec/functions/like.h +++ b/be/src/vec/functions/like.h @@ -115,18 +115,12 @@ struct LikeSearchState { using LikeFn = std::function; -using LikePredicateFn = std::function&, const StringRef&, - ColumnUInt8::Container&, uint16_t* sel, size_t sz)>; - using ScalarLikeFn = std::function; struct LikeState { LikeSearchState search_state; LikeFn function; - // Two functions below are used only for predicate. - LikePredicateFn predicate_like_function; ScalarLikeFn scalar_function; }; @@ -173,48 +167,6 @@ class FunctionLikeBase : public IFunction { static Status regexp_fn(LikeSearchState* state, const ColumnString& val, const StringRef& pattern, ColumnUInt8::Container& result); - // These functions below are used only for predicate. - static Status constant_regex_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, const uint16_t* sel, - size_t sz); - - static Status regexp_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, ColumnUInt8::Container& result, - const uint16_t* sel, size_t sz); - - static Status constant_allpass_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, const uint16_t* sel, - size_t sz); - - static Status constant_starts_with_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, - const uint16_t* sel, size_t sz); - - static Status constant_ends_with_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, - const uint16_t* sel, size_t sz); - - static Status constant_equals_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, const uint16_t* sel, - size_t sz); - - static Status constant_substring_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, - ColumnUInt8::Container& result, - const uint16_t* sel, size_t sz); - static Status constant_allpass_fn_scalar(LikeSearchState* state, const StringRef& val, const StringRef& pattern, unsigned char* result); @@ -257,11 +209,6 @@ class FunctionLike : public FunctionLikeBase { static Status like_fn(LikeSearchState* state, const ColumnString& val, const StringRef& pattern, ColumnUInt8::Container& result); - static Status like_fn_predicate(LikeSearchState* state, - const PredicateColumnType& val, - const StringRef& pattern, ColumnUInt8::Container& result, - uint16_t* sel, size_t sz); - static Status like_fn_scalar(LikeSearchState* state, const StringRef& val, const StringRef& pattern, unsigned char* result); diff --git a/be/src/vec/olap/block_reader.cpp b/be/src/vec/olap/block_reader.cpp index 80fffd44012105f..2342c9faf9144bd 100644 --- a/be/src/vec/olap/block_reader.cpp +++ b/be/src/vec/olap/block_reader.cpp @@ -503,15 +503,4 @@ bool BlockReader::_get_next_row_same() { } } -ColumnPredicate* BlockReader::_parse_to_predicate(const FunctionFilter& function_filter) { - int32_t index = _tablet_schema->field_index(function_filter._col_name); - if (index < 0) { - return nullptr; - } - - // currently only support like predicate - return new LikeColumnPredicate(function_filter._opposite, index, function_filter._fn_ctx, - function_filter._string_param); -} - } // namespace doris::vectorized diff --git a/be/src/vec/olap/block_reader.h b/be/src/vec/olap/block_reader.h index 0fe188419ec0d7f..818440994514cc6 100644 --- a/be/src/vec/olap/block_reader.h +++ b/be/src/vec/olap/block_reader.h @@ -57,8 +57,6 @@ class BlockReader final : public TabletReader { return _vcollect_iter.update_profile(profile); } - ColumnPredicate* _parse_to_predicate(const FunctionFilter& function_filter) override; - private: // Directly read row from rowset and pass to upper caller. No need to do aggregation. // This is usually used for DUPLICATE KEY tables diff --git a/be/src/vec/olap/olap_data_convertor.cpp b/be/src/vec/olap/olap_data_convertor.cpp index a4965e842184870..181f1cd477212fa 100644 --- a/be/src/vec/olap/olap_data_convertor.cpp +++ b/be/src/vec/olap/olap_data_convertor.cpp @@ -22,6 +22,8 @@ // IWYU pragma: no_include #include "common/compiler_util.h" // IWYU pragma: keep #include "common/config.h" +#include "common/exception.h" +#include "common/status.h" #include "olap/hll.h" #include "olap/olap_common.h" #include "olap/tablet_schema.h" @@ -189,6 +191,10 @@ void OlapBlockDataConvertor::set_source_content(const vectorized::Block* block, block->columns() == _convertors.size()); size_t cid = 0; for (const auto& typed_column : *block) { + if (typed_column.column->size() != block->rows()) { + throw Exception(ErrorCode::INTERNAL_ERROR, "input invalid block, block={}", + block->dump_structure()); + } _convertors[cid]->set_source_column(typed_column, row_pos, num_rows); ++cid; } @@ -925,58 +931,34 @@ Status OlapBlockDataConvertor::OlapColumnDataConvertorArray::convert_to_olap() { assert(column_array); assert(data_type_array); - return convert_to_olap(_nullmap, column_array, data_type_array); + return convert_to_olap(column_array, data_type_array); } Status OlapBlockDataConvertor::OlapColumnDataConvertorArray::convert_to_olap( - const UInt8* null_map, const ColumnArray* column_array, - const DataTypeArray* data_type_array) { - const UInt8* item_null_map = nullptr; + const ColumnArray* column_array, const DataTypeArray* data_type_array) { ColumnPtr item_data = column_array->get_data_ptr(); - if (column_array->get_data().is_nullable()) { - const auto& data_nullable_column = - assert_cast(column_array->get_data()); - item_null_map = data_nullable_column.get_null_map_data().data(); - item_data = data_nullable_column.get_nested_column_ptr(); - } - - const auto& offsets = column_array->get_offsets(); - int64_t start_index = _row_pos - 1; - int64_t end_index = _row_pos + _num_rows - 1; - auto start = offsets[start_index]; - auto size = offsets[end_index] - start; - - ColumnWithTypeAndName item_typed_column = { - item_data, remove_nullable(data_type_array->get_nested_type()), ""}; - _item_convertor->set_source_column(item_typed_column, start, size); - RETURN_IF_ERROR(_item_convertor->convert_to_olap()); - - CollectionValue* collection_value = _values.data(); - for (size_t i = 0; i < _num_rows; ++i, ++collection_value) { - int64_t cur_pos = _row_pos + i; - int64_t prev_pos = cur_pos - 1; - if (_nullmap && _nullmap[cur_pos]) { - continue; - } - auto offset = offsets[prev_pos]; - auto size = offsets[cur_pos] - offsets[prev_pos]; - new (collection_value) CollectionValue(size); - if (size == 0) { - continue; - } + auto start_offset = column_array->offset_at(_row_pos); + auto end_offset = column_array->offset_at(_row_pos + _num_rows); + auto elem_size = end_offset - start_offset; - if (column_array->get_data().is_nullable()) { - collection_value->set_has_null(true); - collection_value->set_null_signs( - const_cast(reinterpret_cast(item_null_map + offset))); - } - // get_data_at should use offset - offsets[start_index] since - // start_index may be changed after OlapColumnDataConvertorArray::set_source_column. - // Using just offset may access the memory out of _item_convertor's data range, - collection_value->set_data( - const_cast(_item_convertor->get_data_at(offset - offsets[start_index]))); + _offsets.clear(); + _offsets.reserve(_num_rows + 1); + for (int i = 0; i <= _num_rows; ++i) { + _offsets.push_back(column_array->offset_at(i + _row_pos) - start_offset + _base_offset); } + + _base_offset += elem_size; + + ColumnWithTypeAndName item_typed_column = {item_data, data_type_array->get_nested_type(), + "array.item"}; + _item_convertor->set_source_column(item_typed_column, start_offset, elem_size); + RETURN_IF_ERROR(_item_convertor->convert_to_olap()); + + _results[0] = (void*)elem_size; + _results[1] = _offsets.data(); + _results[2] = _item_convertor->get_data(); + _results[3] = _item_convertor->get_nullmap(); return Status::OK(); } diff --git a/be/src/vec/olap/olap_data_convertor.h b/be/src/vec/olap/olap_data_convertor.h index 320e31fa52d89fd..3f8e0d497f6b4e9 100644 --- a/be/src/vec/olap/olap_data_convertor.h +++ b/be/src/vec/olap/olap_data_convertor.h @@ -429,18 +429,30 @@ class OlapBlockDataConvertor { std::vector _results; }; - class OlapColumnDataConvertorArray - : public OlapColumnDataConvertorPaddedPODArray { + class OlapColumnDataConvertorArray : public OlapColumnDataConvertorBase { public: OlapColumnDataConvertorArray(OlapColumnDataConvertorBaseUPtr item_convertor) - : _item_convertor(std::move(item_convertor)) {} + : _item_convertor(std::move(item_convertor)) { + _base_offset = 0; + _results.resize(4); // size + offset + item_data + item_nullmap + } + const void* get_data() const override { return _results.data(); }; + const void* get_data_at(size_t offset) const override { + LOG(FATAL) << "now not support get_data_at for OlapColumnDataConvertorArray"; + }; Status convert_to_olap() override; private: - Status convert_to_olap(const UInt8* null_map, const ColumnArray* column_array, + // Status convert_to_olap(const UInt8* null_map, const ColumnArray* column_array, + // const DataTypeArray* data_type_array); + Status convert_to_olap(const ColumnArray* column_array, const DataTypeArray* data_type_array); OlapColumnDataConvertorBaseUPtr _item_convertor; + UInt64 _base_offset; + PaddedPODArray _offsets; // array offsets in disk layout + // size + offsets_data + item_data + item_nullmap + std::vector _results; }; class OlapColumnDataConvertorMap : public OlapColumnDataConvertorBase { diff --git a/be/src/vec/olap/vcollect_iterator.cpp b/be/src/vec/olap/vcollect_iterator.cpp index 78fad44fb73c18c..a85f526d653e1c3 100644 --- a/be/src/vec/olap/vcollect_iterator.cpp +++ b/be/src/vec/olap/vcollect_iterator.cpp @@ -58,11 +58,7 @@ namespace vectorized { } \ } while (false) -VCollectIterator::~VCollectIterator() { - for (auto child : _children) { - delete child; - } -} +VCollectIterator::~VCollectIterator() = default; void VCollectIterator::init(TabletReader* reader, bool ori_data_overlapping, bool force_merge, bool is_reverse) { @@ -102,8 +98,7 @@ Status VCollectIterator::add_child(const RowSetSplits& rs_splits) { return Status::OK(); } - std::unique_ptr child(new Level0Iterator(rs_splits.rs_reader, _reader)); - _children.push_back(child.release()); + _children.push_back(std::make_unique(rs_splits.rs_reader, _reader)); return Status::OK(); } @@ -127,7 +122,6 @@ Status VCollectIterator::build_heap(std::vector& rs_reade c_iter != _children.end();) { auto s = (*c_iter)->init(have_multiple_child); if (!s.ok()) { - delete (*c_iter); c_iter = _children.erase(c_iter); r_iter = rs_readers.erase(r_iter); if (!s.is()) { @@ -156,39 +150,38 @@ Status VCollectIterator::build_heap(std::vector& rs_reade auto base_reader_child = _children.begin(); std::advance(base_reader_child, base_reader_idx); - std::list cumu_children; + std::list> cumu_children; for (auto iter = _children.begin(); iter != _children.end();) { if (iter != base_reader_child) { - cumu_children.push_back(*iter); + cumu_children.push_back(std::move(*iter)); iter = _children.erase(iter); } else { ++iter; } } - auto cumu_iter = std::make_unique( - cumu_children, _reader, cumu_children.size() > 1, _is_reverse, _skip_same); + bool is_merge = cumu_children.size() > 1; + std::unique_ptr cumu_iter = std::make_unique( + std::move(cumu_children), _reader, is_merge, _is_reverse, _skip_same); RETURN_IF_NOT_EOF_AND_OK(cumu_iter->init()); - std::list children; - children.push_back(*base_reader_child); - children.push_back(cumu_iter.get()); - auto level1_iter = - new Level1Iterator(children, _reader, _merge, _is_reverse, _skip_same); - cumu_iter.release(); - _inner_iter.reset(level1_iter); + std::list> children; + children.push_back(std::move(*base_reader_child)); + children.push_back(std::move(cumu_iter)); + _inner_iter.reset(new Level1Iterator(std::move(children), _reader, _merge, _is_reverse, + _skip_same)); // need to clear _children here, or else if the following _inner_iter->init() return early // base_reader_child will be double deleted _children.clear(); } else { // _children.size() == 1 - _inner_iter.reset( - new Level1Iterator(_children, _reader, _merge, _is_reverse, _skip_same)); + _inner_iter.reset(new Level1Iterator(std::move(_children), _reader, _merge, _is_reverse, + _skip_same)); } } else { - auto level1_iter = std::make_unique(_children, _reader, _merge, _is_reverse, - _skip_same); + auto level1_iter = std::make_unique(std::move(_children), _reader, _merge, + _is_reverse, _skip_same); _children.clear(); RETURN_IF_ERROR(level1_iter->init_level0_iterators_for_union()); - _inner_iter.reset(level1_iter.release()); + _inner_iter = std::move(level1_iter); } RETURN_IF_NOT_EOF_AND_OK(_inner_iter->init()); // Clear _children earlier to release any related references @@ -592,10 +585,10 @@ Status VCollectIterator::Level0Iterator::current_block_row_locations( } VCollectIterator::Level1Iterator::Level1Iterator( - const std::list& children, TabletReader* reader, + std::list> children, TabletReader* reader, bool merge, bool is_reverse, bool skip_same) : LevelIterator(reader), - _children(children), + _children(std::move(children)), _reader(reader), _merge(merge), _is_reverse(is_reverse), @@ -608,20 +601,10 @@ VCollectIterator::Level1Iterator::Level1Iterator( } VCollectIterator::Level1Iterator::~Level1Iterator() { - for (auto child : _children) { - if (child != nullptr) { - delete child; - child = nullptr; - } - } - if (_heap) { while (!_heap->empty()) { - auto child = _heap->top(); + delete _heap->top(); _heap->pop(); - if (child) { - delete child; - } } } } @@ -681,18 +664,20 @@ Status VCollectIterator::Level1Iterator::init(bool get_data_by_ref) { } } _heap.reset(new MergeHeap {LevelIteratorComparator(sequence_loc, _is_reverse)}); - for (auto child : _children) { + for (auto&& child : _children) { DCHECK(child != nullptr); //DCHECK(child->current_row().ok()); - _heap->push(child); + _heap->push(child.release()); } - _cur_child = _heap->top(); + _cur_child.reset(_heap->top()); + _heap->pop(); // Clear _children earlier to release any related references _children.clear(); } else { _merge = false; _heap.reset(nullptr); - _cur_child = *_children.begin(); + _cur_child = std::move(*_children.begin()); + _children.pop_front(); } _ref = *_cur_child->current_row_ref(); return Status::OK(); @@ -704,7 +689,6 @@ Status VCollectIterator::Level1Iterator::init_level0_iterators_for_union() { for (auto iter = _children.begin(); iter != _children.end();) { auto s = (*iter)->init_for_union(is_first_child, have_multiple_child); if (!s.ok()) { - delete (*iter); iter = _children.erase(iter); if (!s.is()) { return s; @@ -720,24 +704,24 @@ Status VCollectIterator::Level1Iterator::init_level0_iterators_for_union() { } Status VCollectIterator::Level1Iterator::_merge_next(IteratorRowRef* ref) { - _heap->pop(); auto res = _cur_child->next(ref); if (LIKELY(res.ok())) { - _heap->push(_cur_child); - _cur_child = _heap->top(); + _heap->push(_cur_child.release()); + _cur_child.reset(_heap->top()); + _heap->pop(); } else if (res.is()) { // current child has been read, to read next - delete _cur_child; if (!_heap->empty()) { - _cur_child = _heap->top(); + _cur_child.reset(_heap->top()); + _heap->pop(); } else { _ref.reset(); - _cur_child = nullptr; + _cur_child.reset(); return Status::Error(""); } } else { _ref.reset(); - _cur_child = nullptr; + _cur_child.reset(); LOG(WARNING) << "failed to get next from child, res=" << res; return res; } @@ -763,17 +747,16 @@ Status VCollectIterator::Level1Iterator::_normal_next(IteratorRowRef* ref) { return Status::OK(); } else if (res.is()) { // current child has been read, to read next - delete _cur_child; - _children.pop_front(); if (!_children.empty()) { - _cur_child = *(_children.begin()); + _cur_child = std::move(*(_children.begin())); + _children.pop_front(); return _normal_next(ref); } else { - _cur_child = nullptr; + _cur_child.reset(); return Status::Error(""); } } else { - _cur_child = nullptr; + _cur_child.reset(); LOG(WARNING) << "failed to get next from child, res=" << res; return res; } @@ -861,17 +844,16 @@ Status VCollectIterator::Level1Iterator::_normal_next(Block* block) { return Status::OK(); } else if (res.is()) { // current child has been read, to read next - delete _cur_child; - _children.pop_front(); if (!_children.empty()) { - _cur_child = *(_children.begin()); + _cur_child = std::move(*(_children.begin())); + _children.pop_front(); return _normal_next(block); } else { - _cur_child = nullptr; + _cur_child.reset(); return Status::Error(""); } } else { - _cur_child = nullptr; + _cur_child.reset(); LOG(WARNING) << "failed to get next from child, res=" << res; return res; } diff --git a/be/src/vec/olap/vcollect_iterator.h b/be/src/vec/olap/vcollect_iterator.h index 7faa47ac2b87965..3fa984928946192 100644 --- a/be/src/vec/olap/vcollect_iterator.h +++ b/be/src/vec/olap/vcollect_iterator.h @@ -258,8 +258,8 @@ class VCollectIterator { // Iterate from LevelIterators (maybe Level0Iterators or Level1Iterator or mixed) class Level1Iterator : public LevelIterator { public: - Level1Iterator(const std::list& children, TabletReader* reader, bool merge, - bool is_reverse, bool skip_same); + Level1Iterator(std::list> children, TabletReader* reader, + bool merge, bool is_reverse, bool skip_same); Status init(bool get_data_by_ref = false) override; @@ -295,10 +295,10 @@ class VCollectIterator { // Each LevelIterator corresponds to a rowset reader, // it will be cleared after '_heap' has been initialized when '_merge == true'. - std::list _children; + std::list> _children; // point to the Level0Iterator containing the next output row. // null when VCollectIterator hasn't been initialized or reaches EOF. - LevelIterator* _cur_child = nullptr; + std::unique_ptr _cur_child; TabletReader* _reader = nullptr; // when `_merge == true`, rowset reader returns ordered rows and VCollectIterator uses a priority queue to merge @@ -321,7 +321,7 @@ class VCollectIterator { // Each LevelIterator corresponds to a rowset reader, // it will be cleared after '_inner_iter' has been initialized. - std::list _children; + std::list> _children; bool _merge = true; // reverse the compare order @@ -338,4 +338,4 @@ class VCollectIterator { }; } // namespace vectorized -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/vec/runtime/vdata_stream_mgr.cpp b/be/src/vec/runtime/vdata_stream_mgr.cpp index eeee0723717627e..ad161828f905a67 100644 --- a/be/src/vec/runtime/vdata_stream_mgr.cpp +++ b/be/src/vec/runtime/vdata_stream_mgr.cpp @@ -120,8 +120,9 @@ Status VDataStreamMgr::transmit_block(const PTransmitDataParams* request, bool eos = request->eos(); if (request->has_block()) { - recvr->add_block(request->block(), request->sender_id(), request->be_number(), - request->packet_seq(), eos ? nullptr : done); + RETURN_IF_ERROR(recvr->add_block(request->block(), request->sender_id(), + request->be_number(), request->packet_seq(), + eos ? nullptr : done)); } if (eos) { diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index 0c76df5b9ab1487..cc908d47e08e921 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -108,13 +108,13 @@ Status VDataStreamRecvr::SenderQueue::_inner_get_batch_without_lock(Block* block return Status::OK(); } -void VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_number, - int64_t packet_seq, - ::google::protobuf::Closure** done) { +Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_number, + int64_t packet_seq, + ::google::protobuf::Closure** done) { { std::lock_guard l(_lock); if (_is_cancelled) { - return; + return Status::OK(); } auto iter = _packet_seq_map.find(be_number); if (iter != _packet_seq_map.end()) { @@ -122,7 +122,7 @@ void VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_numbe LOG(WARNING) << fmt::format( "packet already exist [cur_packet_id= {} receive_packet_id={}]", iter->second, packet_seq); - return; + return Status::OK(); } iter->second = packet_seq; } else { @@ -134,7 +134,7 @@ void VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_numbe DCHECK(_num_remaining_senders >= 0); if (_num_remaining_senders == 0) { DCHECK(_sender_eos_set.end() != _sender_eos_set.find(be_number)); - return; + return Status::OK(); } } @@ -142,7 +142,8 @@ void VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_numbe int64_t deserialize_time = 0; { SCOPED_RAW_TIMER(&deserialize_time); - block = Block::create_unique(pblock); + block = Block::create_unique(); + RETURN_IF_ERROR(block->deserialize(pblock)); } auto block_byte_size = block->allocated_bytes(); @@ -150,7 +151,7 @@ void VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_numbe std::lock_guard l(_lock); if (_is_cancelled) { - return; + return Status::OK(); } COUNTER_UPDATE(_recvr->_deserialize_row_batch_timer, deserialize_time); @@ -176,6 +177,7 @@ void VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_numbe if (!empty) { _data_arrival_cv.notify_one(); } + return Status::OK(); } void VDataStreamRecvr::SenderQueue::add_block(Block* block, bool use_move) { @@ -376,11 +378,11 @@ Status VDataStreamRecvr::create_merger(const VExprContextSPtrs& ordering_expr, return Status::OK(); } -void VDataStreamRecvr::add_block(const PBlock& pblock, int sender_id, int be_number, - int64_t packet_seq, ::google::protobuf::Closure** done) { +Status VDataStreamRecvr::add_block(const PBlock& pblock, int sender_id, int be_number, + int64_t packet_seq, ::google::protobuf::Closure** done) { SCOPED_ATTACH_TASK_WITH_ID(_query_mem_tracker, _query_id, _fragment_instance_id); int use_sender_id = _is_merging ? sender_id : 0; - _sender_queues[use_sender_id]->add_block(pblock, be_number, packet_seq, done); + return _sender_queues[use_sender_id]->add_block(pblock, be_number, packet_seq, done); } void VDataStreamRecvr::add_block(Block* block, int sender_id, bool use_move) { diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index eb57c57b0d26d71..fe9910492bf5a94 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -75,8 +75,8 @@ class VDataStreamRecvr { const std::vector& nulls_first, size_t batch_size, int64_t limit, size_t offset); - void add_block(const PBlock& pblock, int sender_id, int be_number, int64_t packet_seq, - ::google::protobuf::Closure** done); + Status add_block(const PBlock& pblock, int sender_id, int be_number, int64_t packet_seq, + ::google::protobuf::Closure** done); void add_block(Block* block, int sender_id, bool use_move); @@ -185,8 +185,8 @@ class VDataStreamRecvr::SenderQueue { virtual Status get_batch(Block* next_block, bool* eos); - void add_block(const PBlock& pblock, int be_number, int64_t packet_seq, - ::google::protobuf::Closure** done); + Status add_block(const PBlock& pblock, int be_number, int64_t packet_seq, + ::google::protobuf::Closure** done); virtual void add_block(Block* block, bool use_move); diff --git a/be/src/vec/runtime/vfile_writer_wrapper.h b/be/src/vec/runtime/vfile_writer_wrapper.h new file mode 100644 index 000000000000000..e418a14ffac6450 --- /dev/null +++ b/be/src/vec/runtime/vfile_writer_wrapper.h @@ -0,0 +1,51 @@ +// 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 "common/status.h" +#include "vec/core/block.h" +#include "vec/exprs/vexpr_fwd.h" + +namespace doris::vectorized { + +class VFileWriterWrapper { +public: + VFileWriterWrapper(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) {} + + virtual ~VFileWriterWrapper() = default; + + virtual Status prepare() = 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; +}; +} // namespace doris::vectorized diff --git a/be/src/vec/runtime/vorc_writer.cpp b/be/src/vec/runtime/vorc_writer.cpp index 34f274cc0622457..df9615d66895907 100644 --- a/be/src/vec/runtime/vorc_writer.cpp +++ b/be/src/vec/runtime/vorc_writer.cpp @@ -171,7 +171,7 @@ Status VOrcWriterWrapper::close() { RETURN_WRONG_TYPE \ } -#define WRITE_LARGEINT_STRING_INTO_BATCH(VECTOR_BATCH, COLUMN) \ +#define WRITE_LARGEINT_STRING_INTO_BATCH(VECTOR_BATCH, COLUMN, BUFFER) \ VECTOR_BATCH* cur_batch = dynamic_cast(root->fields[i]); \ const size_t begin_off = offset; \ if (null_map != nullptr) { \ @@ -185,14 +185,14 @@ Status VOrcWriterWrapper::close() { 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; \ + 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()); \ + strcpy(const_cast(BUFFER.data) + offset, value_str.c_str()); \ offset += len; \ cur_batch->length[row_id] = len; \ } \ @@ -202,7 +202,7 @@ Status VOrcWriterWrapper::close() { 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; \ + cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ data_off += cur_batch->length[row_id]; \ } \ } \ @@ -211,27 +211,27 @@ Status VOrcWriterWrapper::close() { 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; \ + 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()); \ + 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 = 0; row_id < sz; row_id++) { \ - cur_batch->data[row_id] = const_cast(buffer.data) + begin_off + data_off; \ + 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) \ +#define WRITE_DATE_STRING_INTO_BATCH(FROM, TO, BUFFER) \ orc::StringVectorBatch* cur_batch = dynamic_cast(root->fields[i]); \ const size_t begin_off = offset; \ if (null_map != nullptr) { \ @@ -244,13 +244,13 @@ Status VOrcWriterWrapper::close() { 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; \ + .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; \ @@ -261,7 +261,7 @@ Status VOrcWriterWrapper::close() { 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; \ + cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ data_off += cur_batch->length[row_id]; \ } \ } \ @@ -269,27 +269,27 @@ Status VOrcWriterWrapper::close() { check_and_get_column>(col)) { \ for (size_t row_id = 0; row_id < sz; 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; \ + .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 = 0; row_id < sz; row_id++) { \ - cur_batch->data[row_id] = const_cast(buffer.data) + begin_off + data_off; \ + 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) \ +#define WRITE_DATETIMEV2_STRING_INTO_BATCH(FROM, TO, BUFFER) \ orc::StringVectorBatch* cur_batch = dynamic_cast(root->fields[i]); \ const size_t begin_off = offset; \ if (null_map != nullptr) { \ @@ -304,13 +304,13 @@ Status VOrcWriterWrapper::close() { 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; \ + .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; \ @@ -321,7 +321,7 @@ Status VOrcWriterWrapper::close() { 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; \ + cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ data_off += cur_batch->length[row_id]; \ } \ } \ @@ -330,20 +330,20 @@ Status VOrcWriterWrapper::close() { for (size_t row_id = 0; row_id < sz; row_id++) { \ int output_scale = _output_vexpr_ctxs[i]->root()->type().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; \ + .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 = 0; row_id < sz; row_id++) { \ - cur_batch->data[row_id] = const_cast(buffer.data) + begin_off + data_off; \ + cur_batch->data[row_id] = const_cast(BUFFER.data) + begin_off + data_off; \ data_off += cur_batch->length[row_id]; \ } \ } else { \ @@ -404,9 +404,14 @@ Status VOrcWriterWrapper::write(const Block& block) { } // Buffer used by date/datetime/datev2/datetimev2/largeint type - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - StringRef buffer(ptr, BUFFER_UNIT_SIZE); - size_t offset = 0; + std::vector bufferList(block.columns()); + Defer defer {[&]() { + for (auto& bufferRef : bufferList) { + if (bufferRef.data) { + free(const_cast(bufferRef.data)); + } + } + }}; size_t sz = block.rows(); auto row_batch = _create_row_batch(sz); @@ -455,7 +460,12 @@ Status VOrcWriterWrapper::write(const Block& block) { break; } case TYPE_LARGEINT: { - WRITE_LARGEINT_STRING_INTO_BATCH(orc::StringVectorBatch, ColumnVector) + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); + bufferList[i].data = ptr; + bufferList[i].size = BUFFER_UNIT_SIZE; + size_t offset = 0; + WRITE_LARGEINT_STRING_INTO_BATCH(orc::StringVectorBatch, ColumnVector, + bufferList[i]) SET_NUM_ELEMENTS; break; } @@ -472,17 +482,30 @@ Status VOrcWriterWrapper::write(const Block& block) { } case TYPE_DATETIME: case TYPE_DATE: { - WRITE_DATE_STRING_INTO_BATCH(Int64, VecDateTimeValue) + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); + bufferList[i].data = ptr; + bufferList[i].size = BUFFER_UNIT_SIZE; + size_t offset = 0; + WRITE_DATE_STRING_INTO_BATCH(Int64, VecDateTimeValue, bufferList[i]) SET_NUM_ELEMENTS break; } case TYPE_DATEV2: { - WRITE_DATE_STRING_INTO_BATCH(UInt32, DateV2Value) + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); + bufferList[i].data = ptr; + bufferList[i].size = BUFFER_UNIT_SIZE; + size_t offset = 0; + WRITE_DATE_STRING_INTO_BATCH(UInt32, DateV2Value, bufferList[i]) SET_NUM_ELEMENTS break; } case TYPE_DATETIMEV2: { - WRITE_DATETIMEV2_STRING_INTO_BATCH(UInt64, DateV2Value) + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); + bufferList[i].data = ptr; + bufferList[i].size = BUFFER_UNIT_SIZE; + size_t offset = 0; + WRITE_DATETIMEV2_STRING_INTO_BATCH(UInt64, DateV2Value, + bufferList[i]) SET_NUM_ELEMENTS break; } @@ -594,11 +617,9 @@ Status VOrcWriterWrapper::write(const Block& block) { return Status::InternalError(e.what()); } root->numElements = sz; - _writer->add(*row_batch); _cur_written_rows += sz; - free(const_cast(buffer.data)); return Status::OK(); } diff --git a/be/src/vec/runtime/vparquet_writer.h b/be/src/vec/runtime/vparquet_writer.h index 22410b5d069a9d6..36514c9fe8ecb26 100644 --- a/be/src/vec/runtime/vparquet_writer.h +++ b/be/src/vec/runtime/vparquet_writer.h @@ -26,12 +26,7 @@ #include #include -#include -#include - -#include "common/status.h" -#include "vec/core/block.h" -#include "vec/exprs/vexpr_fwd.h" +#include "vfile_writer_wrapper.h" namespace doris { namespace io { @@ -90,29 +85,6 @@ class ParquetBuildHelper { const TypeDescriptor& type_desc); }; -class VFileWriterWrapper { -public: - VFileWriterWrapper(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) {} - - virtual ~VFileWriterWrapper() = default; - - virtual Status prepare() = 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; -}; - // a wrapper of parquet output stream class VParquetWriterWrapper final : public VFileWriterWrapper { public: diff --git a/be/src/vec/runtime/vsorted_run_merger.cpp b/be/src/vec/runtime/vsorted_run_merger.cpp index 5f4b8206c7691fa..1bdd82ba8e4f685 100644 --- a/be/src/vec/runtime/vsorted_run_merger.cpp +++ b/be/src/vec/runtime/vsorted_run_merger.cpp @@ -20,6 +20,8 @@ #include #include +#include "common/exception.h" +#include "common/status.h" #include "util/runtime_profile.h" #include "util/stopwatch.hpp" #include "vec/columns/column.h" @@ -161,6 +163,13 @@ Status VSortedRunMerger::get_next(Block* output_block, bool* eos) { VectorizedUtils::build_mutable_mem_reuse_block(output_block, _empty_block); MutableColumns& merged_columns = m_block.mutable_columns(); + if (num_columns != merged_columns.size()) { + throw Exception( + ErrorCode::INTERNAL_ERROR, + "num_columns!=merged_columns.size(), num_columns={}, merged_columns.size()={}", + num_columns, merged_columns.size()); + } + /// Take rows from queue in right order and push to 'merged'. size_t merged_rows = 0; while (!_priority_queue.empty()) { @@ -170,8 +179,9 @@ Status VSortedRunMerger::get_next(Block* output_block, bool* eos) { if (_offset > 0) { _offset--; } else { - for (size_t i = 0; i < num_columns; ++i) + for (size_t i = 0; i < num_columns; ++i) { merged_columns[i]->insert_from(*current->all_columns[i], current->pos); + } ++merged_rows; } diff --git a/be/src/vec/sink/vtable_sink.h b/be/src/vec/sink/async_writer_sink.h similarity index 75% rename from be/src/vec/sink/vtable_sink.h rename to be/src/vec/sink/async_writer_sink.h index fc41faa175f7066..8eb177ce8eedfef 100644 --- a/be/src/vec/sink/vtable_sink.h +++ b/be/src/vec/sink/async_writer_sink.h @@ -42,10 +42,10 @@ namespace vectorized { class Block; template -class VTableSink : public DataSink { +class AsyncWriterSink : public DataSink { public: - VTableSink(ObjectPool* pool, const RowDescriptor& row_desc, const std::vector& t_exprs) - : DataSink(row_desc), _pool(pool), _t_output_expr(t_exprs) { + AsyncWriterSink(const RowDescriptor& row_desc, const std::vector& t_exprs) + : DataSink(row_desc), _t_output_expr(t_exprs) { _name = Name; } @@ -87,14 +87,12 @@ class VTableSink : public DataSink { return _writer->sink(block, eos); } - RuntimeProfile* profile() override { return _profile; } - bool can_write() override { return _writer->can_write(); } Status close(RuntimeState* state, Status exec_status) override { if (_writer->need_normal_close()) { if (exec_status.ok() && !state->is_cancelled()) { - RETURN_IF_ERROR(_writer->finish_trans()); + RETURN_IF_ERROR(_writer->commit_trans()); } RETURN_IF_ERROR(_writer->close()); } @@ -111,37 +109,31 @@ class VTableSink : public DataSink { bool is_close_done() override { return !_writer->is_pending_finish(); } protected: - // owned by RuntimeState - ObjectPool* _pool; const std::vector& _t_output_expr; VExprContextSPtrs _output_vexpr_ctxs; std::unique_ptr _writer; - RuntimeProfile* _profile; }; inline constexpr char VJDBC_TABLE_SINK_NAME[] = "VJdbcTableSink"; inline constexpr char VODBC_TABLE_SINK_NAME[] = "VOdbcTableSink"; inline constexpr char VMYSQL_TABLE_SINK_NAME[] = "VMysqlTableSink"; -class VJdbcTableSink : public VTableSink { +class VJdbcTableSink : public AsyncWriterSink { public: - VJdbcTableSink(ObjectPool* pool, const RowDescriptor& row_desc, - const std::vector& t_exprs) - : VTableSink(pool, row_desc, t_exprs) {}; + VJdbcTableSink(const RowDescriptor& row_desc, const std::vector& t_exprs) + : AsyncWriterSink(row_desc, t_exprs) {}; }; -class VOdbcTableSink : public VTableSink { +class VOdbcTableSink : public AsyncWriterSink { public: - VOdbcTableSink(ObjectPool* pool, const RowDescriptor& row_desc, - const std::vector& t_exprs) - : VTableSink(pool, row_desc, t_exprs) {}; + VOdbcTableSink(const RowDescriptor& row_desc, const std::vector& t_exprs) + : AsyncWriterSink(row_desc, t_exprs) {}; }; -class VMysqlTableSink : public VTableSink { +class VMysqlTableSink : public AsyncWriterSink { public: - VMysqlTableSink(ObjectPool* pool, const RowDescriptor& row_desc, - const std::vector& t_exprs) - : VTableSink(pool, row_desc, t_exprs) {}; + VMysqlTableSink(const RowDescriptor& row_desc, const std::vector& t_exprs) + : AsyncWriterSink(row_desc, t_exprs) {}; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/sink/load_stream_stub.cpp b/be/src/vec/sink/load_stream_stub.cpp new file mode 100644 index 000000000000000..1fbc8228b21f8a3 --- /dev/null +++ b/be/src/vec/sink/load_stream_stub.cpp @@ -0,0 +1,232 @@ +// 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/load_stream_stub.h" + +#include "olap/rowset/rowset_writer.h" +#include "util/brpc_client_cache.h" +#include "util/network_util.h" +#include "util/thrift_util.h" + +namespace doris { + +int LoadStreamStub::LoadStreamReplyHandler::on_received_messages(brpc::StreamId id, + butil::IOBuf* const messages[], + size_t size) { + for (size_t i = 0; i < size; i++) { + butil::IOBufAsZeroCopyInputStream wrapper(*messages[i]); + PWriteStreamSinkResponse response; + response.ParseFromZeroCopyStream(&wrapper); + + Status st = Status::create(response.status()); + + std::stringstream ss; + ss << "received response from backend " << _stub->_dst_id; + if (response.success_tablet_ids_size() > 0) { + ss << ", success tablet ids:"; + for (auto tablet_id : response.success_tablet_ids()) { + ss << " " << tablet_id; + } + std::lock_guard lock(_stub->_success_tablets_mutex); + for (auto tablet_id : response.success_tablet_ids()) { + _stub->_success_tablets.push_back(tablet_id); + } + } + if (response.failed_tablet_ids_size() > 0) { + ss << ", failed tablet ids:"; + for (auto tablet_id : response.failed_tablet_ids()) { + ss << " " << tablet_id; + } + std::lock_guard lock(_stub->_failed_tablets_mutex); + for (auto tablet_id : response.failed_tablet_ids()) { + _stub->_failed_tablets.push_back(tablet_id); + } + } + ss << ", status: " << st; + LOG(INFO) << ss.str(); + + if (response.has_load_stream_profile()) { + TRuntimeProfileTree tprofile; + const uint8_t* buf = + reinterpret_cast(response.load_stream_profile().data()); + uint32_t len = response.load_stream_profile().size(); + auto status = deserialize_thrift_msg(buf, &len, false, &tprofile); + if (status.ok()) { + // TODO + //_sink->_state->load_channel_profile()->update(tprofile); + } else { + LOG(WARNING) << "load stream TRuntimeProfileTree deserialize failed, errmsg=" + << status; + } + } + } + return 0; +} + +void LoadStreamStub::LoadStreamReplyHandler::on_closed(brpc::StreamId id) { + std::lock_guard lock(_stub->_mutex); + _stub->_is_closed = true; + _stub->_close_cv.notify_all(); +} + +LoadStreamStub::LoadStreamStub(PUniqueId load_id, int64_t src_id) + : _load_id(load_id), + _src_id(src_id), + _tablet_schema_for_index(std::make_shared()), + _enable_unique_mow_for_index(std::make_shared()) {}; + +LoadStreamStub::LoadStreamStub(LoadStreamStub& stub) + : _load_id(stub._load_id), + _src_id(stub._src_id), + _tablet_schema_for_index(stub._tablet_schema_for_index), + _enable_unique_mow_for_index(stub._enable_unique_mow_for_index) {}; + +LoadStreamStub::~LoadStreamStub() { + std::unique_lock lock(_mutex); + if (_is_init && !_is_closed) { + brpc::StreamClose(_stream_id); + } +} + +// open_stream_sink +// tablets means +Status LoadStreamStub::open(BrpcClientCache* client_cache, + const NodeInfo& node_info, int64_t txn_id, + const OlapTableSchemaParam& schema, + const std::vector& tablets_for_schema, bool enable_profile) { + std::unique_lock lock(_mutex); + if (_is_init) { + return Status::OK(); + } + _dst_id = node_info.id; + std::string host_port = get_host_port(node_info.host, node_info.brpc_port); + brpc::StreamOptions opt; + opt.max_buf_size = 20 << 20; // 20MB + opt.idle_timeout_ms = 30000; + opt.messages_in_batch = 128; + opt.handler = &_handler; + brpc::Controller cntl; + if (int ret = StreamCreate(&_stream_id, cntl, &opt)) { + return Status::Error(ret, "Failed to create stream"); + } + cntl.set_timeout_ms(config::open_stream_sink_timeout_ms); + POpenStreamSinkRequest request; + *request.mutable_load_id() = _load_id; + request.set_src_id(_src_id); + request.set_txn_id(txn_id); + request.set_enable_profile(enable_profile); + schema.to_protobuf(request.mutable_schema()); + for (auto& tablet : tablets_for_schema) { + *request.add_tablets() = tablet; + } + POpenStreamSinkResponse response; + // use "pooled" connection to avoid conflicts between streaming rpc and regular rpc, + // see: https://github.com/apache/brpc/issues/392 + const auto& stub = client_cache->get_new_client_no_cache(host_port, "baidu_std", "pooled"); + stub->open_stream_sink(&cntl, &request, &response, nullptr); + for (const auto& resp : response.tablet_schemas()) { + auto tablet_schema = std::make_unique(); + tablet_schema->init_from_pb(resp.tablet_schema()); + _tablet_schema_for_index->emplace(resp.index_id(), std::move(tablet_schema)); + _enable_unique_mow_for_index->emplace(resp.index_id(), + resp.enable_unique_key_merge_on_write()); + } + if (cntl.Failed()) { + return Status::InternalError("Failed to connect to backend {}: {}", _dst_id, + cntl.ErrorText()); + } + LOG(INFO) << "Opened stream " << _stream_id << " for backend " << _dst_id << " (" << host_port + << ")"; + _is_init = true; + return Status::OK(); +} + +// APPEND_DATA +Status LoadStreamStub::append_data(int64_t partition_id, int64_t index_id, int64_t tablet_id, + int64_t segment_id, std::span data, + bool segment_eos) { + PStreamHeader header; + header.set_src_id(_src_id); + *header.mutable_load_id() = _load_id; + header.set_partition_id(partition_id); + header.set_index_id(index_id); + header.set_tablet_id(tablet_id); + header.set_segment_id(segment_id); + header.set_segment_eos(segment_eos); + header.set_opcode(doris::PStreamHeader::APPEND_DATA); + return _encode_and_send(header, data); +} + +// ADD_SEGMENT +Status LoadStreamStub::add_segment(int64_t partition_id, int64_t index_id, int64_t tablet_id, + int64_t segment_id, SegmentStatistics& segment_stat) { + PStreamHeader header; + header.set_src_id(_src_id); + *header.mutable_load_id() = _load_id; + header.set_partition_id(partition_id); + header.set_index_id(index_id); + header.set_tablet_id(tablet_id); + header.set_segment_id(segment_id); + header.set_opcode(doris::PStreamHeader::ADD_SEGMENT); + segment_stat.to_pb(header.mutable_segment_statistics()); + return _encode_and_send(header); +} + +// CLOSE_LOAD +Status LoadStreamStub::close_load(const std::vector& tablets_to_commit) { + PStreamHeader header; + *header.mutable_load_id() = _load_id; + header.set_src_id(_src_id); + header.set_opcode(doris::PStreamHeader::CLOSE_LOAD); + for (const auto& tablet : tablets_to_commit) { + *header.add_tablets_to_commit() = tablet; + } + return _encode_and_send(header); +} + +Status LoadStreamStub::_encode_and_send(PStreamHeader& header, std::span data) { + butil::IOBuf buf; + size_t header_len = header.ByteSizeLong(); + buf.append(reinterpret_cast(&header_len), sizeof(header_len)); + buf.append(header.SerializeAsString()); + for (const auto& slice : data) { + buf.append(slice.get_data(), slice.get_size()); + } + return _send_with_retry(buf); +} + +Status LoadStreamStub::_send_with_retry(butil::IOBuf buf) { + for (;;) { + int ret = brpc::StreamWrite(_stream_id, buf); + switch (ret) { + case 0: + return Status::OK(); + case EAGAIN: { + const timespec time = butil::seconds_from_now(60); + int wait_ret = brpc::StreamWait(_stream_id, &time); + if (wait_ret != 0) { + return Status::InternalError("StreamWait failed, err = ", wait_ret); + } + break; + } + default: + return Status::InternalError("StreamWrite failed, err = {}", ret); + } + } +} + +} // namespace doris diff --git a/be/src/vec/sink/load_stream_stub.h b/be/src/vec/sink/load_stream_stub.h new file mode 100644 index 000000000000000..268de6ca83aba49 --- /dev/null +++ b/be/src/vec/sink/load_stream_stub.h @@ -0,0 +1,192 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +// IWYU pragma: no_include +#include // IWYU pragma: keep +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/status.h" +#include "exec/data_sink.h" +#include "exec/tablet_info.h" +#include "gutil/ref_counted.h" +#include "runtime/exec_env.h" +#include "runtime/memory/mem_tracker.h" +#include "runtime/thread_context.h" +#include "runtime/types.h" +#include "util/countdown_latch.h" +#include "util/runtime_profile.h" +#include "util/stopwatch.hpp" +#include "vec/columns/column.h" +#include "vec/common/allocator.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type.h" +#include "vec/exprs/vexpr_fwd.h" + +namespace doris { +class TabletSchema; +class LoadStreamStub; + +struct SegmentStatistics; + +using IndexToTabletSchema = std::unordered_map>; +using IndexToEnableMoW = std::unordered_map; + +class LoadStreamStub { +private: + class LoadStreamReplyHandler : public brpc::StreamInputHandler { + public: + LoadStreamReplyHandler(LoadStreamStub* stub) : _stub(stub) {} + + int on_received_messages(brpc::StreamId id, butil::IOBuf* const messages[], + size_t size) override; + + void on_idle_timeout(brpc::StreamId id) override {} + + void on_closed(brpc::StreamId id) override; + + private: + LoadStreamStub* _stub; + }; + +public: + // construct new stub + LoadStreamStub(PUniqueId load_id, int64_t src_id); + + // copy constructor, shared_ptr members are shared + LoadStreamStub(LoadStreamStub& stub); + +// for mock this class in UT +#ifdef BE_TEST + virtual +#endif + ~LoadStreamStub(); + + // open_stream_sink + Status open(BrpcClientCache* client_cache, const NodeInfo& node_info, + int64_t txn_id, const OlapTableSchemaParam& schema, + const std::vector& tablets_for_schema, bool enable_profile); + +// for mock this class in UT +#ifdef BE_TEST + virtual +#endif + // APPEND_DATA + Status + append_data(int64_t partition_id, int64_t index_id, int64_t tablet_id, + int64_t segment_id, std::span data, bool segment_eos = false); + + // ADD_SEGMENT + Status add_segment(int64_t partition_id, int64_t index_id, int64_t tablet_id, + int64_t segment_id, SegmentStatistics& segment_stat); + + // CLOSE_LOAD + Status close_load(const std::vector& tablets_to_commit); + + // wait remote to close stream, + // remote will close stream when it receives CLOSE_LOAD + Status close_wait(int64_t timeout_ms = 0) { + std::unique_lock lock(_mutex); + if (!_is_init || _is_closed) { + return Status::OK(); + } + if (timeout_ms > 0) { + int ret = _close_cv.wait_for(lock, timeout_ms * 1000); + return ret == 0 ? Status::OK() : Status::Error(ret, "stream close_wait timeout"); + } + _close_cv.wait(lock); + return Status::OK(); + } + + std::shared_ptr tablet_schema(int64_t index_id) const { + return _tablet_schema_for_index->at(index_id); + } + + bool enable_unique_mow(int64_t index_id) const { + return _enable_unique_mow_for_index->at(index_id); + } + + std::vector success_tablets() { + std::lock_guard lock(_success_tablets_mutex); + return _success_tablets; + } + + std::vector failed_tablets() { + std::lock_guard lock(_failed_tablets_mutex); + return _failed_tablets; + } + + brpc::StreamId stream_id() const { return _stream_id; } + + int64_t src_id() const { return _src_id; } + + int64_t dst_id() const { return _dst_id; } + +private: + Status _encode_and_send(PStreamHeader& header, std::span data = {}); + Status _send_with_retry(butil::IOBuf buf); + +protected: + bool _is_init = false; + bool _is_closed = false; + bthread::Mutex _mutex; + bthread::ConditionVariable _close_cv; + + PUniqueId _load_id; + brpc::StreamId _stream_id; + int64_t _src_id = -1; // source backend_id + int64_t _dst_id = -1; // destination backend_id + LoadStreamReplyHandler _handler {this}; + + bthread::Mutex _success_tablets_mutex; + bthread::Mutex _failed_tablets_mutex; + std::vector _success_tablets; + std::vector _failed_tablets; + + std::shared_ptr _tablet_schema_for_index; + std::shared_ptr _enable_unique_mow_for_index; +}; + +} // namespace doris diff --git a/be/src/vec/sink/multi_cast_data_stream_sink.h b/be/src/vec/sink/multi_cast_data_stream_sink.h index df210d74ffab839..364586cad0dd103 100644 --- a/be/src/vec/sink/multi_cast_data_stream_sink.h +++ b/be/src/vec/sink/multi_cast_data_stream_sink.h @@ -25,7 +25,9 @@ namespace doris::vectorized { class MultiCastDataStreamSink : public DataSink { public: MultiCastDataStreamSink(std::shared_ptr& streamer) - : DataSink(streamer->row_desc()), _multi_cast_data_streamer(streamer) {}; + : DataSink(streamer->row_desc()), _multi_cast_data_streamer(streamer) { + _profile = _multi_cast_data_streamer->profile(); + }; ~MultiCastDataStreamSink() override = default; @@ -39,8 +41,6 @@ class MultiCastDataStreamSink : public DataSink { // use sink to check can_write, now always true after we support spill to disk bool can_write() override { return _multi_cast_data_streamer->can_write(); } - RuntimeProfile* profile() override { return _multi_cast_data_streamer->profile(); } - std::shared_ptr& get_multi_cast_data_streamer() { return _multi_cast_data_streamer; } diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp index fe86eefa4e84c09..23194db6d9dfee0 100644 --- a/be/src/vec/sink/vdata_stream_sender.cpp +++ b/be/src/vec/sink/vdata_stream_sender.cpp @@ -192,7 +192,8 @@ Status Channel::send_block(PBlock* block, bool eos) { { SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); if (enable_http_send_block(_brpc_request, _parent->transfer_large_data_by_brpc())) { - RETURN_IF_ERROR(transmit_block_http(_state, _closure, _brpc_request, _brpc_dest_addr)); + RETURN_IF_ERROR(transmit_block_http(_state->exec_env(), _closure, _brpc_request, + _brpc_dest_addr)); } else { transmit_block(*_brpc_stub, _closure, _brpc_request); } @@ -299,7 +300,6 @@ VDataStreamSender::VDataStreamSender(RuntimeState* state, ObjectPool* pool, int _pool(pool), _current_channel_idx(0), _part_type(sink.output_partition.type), - _profile(nullptr), _serialize_batch_timer(nullptr), _bytes_sent_counter(nullptr), _local_send_timer(nullptr), @@ -364,7 +364,6 @@ VDataStreamSender::VDataStreamSender(RuntimeState* state, ObjectPool* pool, int _pool(pool), _current_channel_idx(0), _part_type(TPartitionType::UNPARTITIONED), - _profile(nullptr), _serialize_batch_timer(nullptr), _compress_timer(nullptr), _brpc_send_timer(nullptr), diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index a7cceb13854e5f6..a3e4ccf2a6abbe4 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -117,7 +117,6 @@ class VDataStreamSender : public DataSink { Status send(RuntimeState* state, Block* block, bool eos = false) override; Status try_close(RuntimeState* state, Status exec_status) override; Status close(RuntimeState* state, Status exec_status) override; - RuntimeProfile* profile() override { return _profile; } RuntimeState* state() { return _state; } @@ -192,7 +191,6 @@ class VDataStreamSender : public DataSink { std::vector*> _channels; std::vector>> _channel_shared_ptrs; - RuntimeProfile* _profile; // Allocated from _pool RuntimeProfile::Counter* _serialize_batch_timer; RuntimeProfile::Counter* _compress_timer; RuntimeProfile::Counter* _brpc_send_timer; diff --git a/be/src/vec/sink/vmemory_scratch_sink.cpp b/be/src/vec/sink/vmemory_scratch_sink.cpp index 6e9f5ca743b71f1..9b65838a3a9d62d 100644 --- a/be/src/vec/sink/vmemory_scratch_sink.cpp +++ b/be/src/vec/sink/vmemory_scratch_sink.cpp @@ -90,6 +90,10 @@ Status MemoryScratchSink::open(RuntimeState* state) { return VExpr::open(_output_vexpr_ctxs, state); } +bool MemoryScratchSink::can_write() { + return _queue->size() < 10; +} + Status MemoryScratchSink::close(RuntimeState* state, Status exec_status) { if (_closed) { return Status::OK(); diff --git a/be/src/vec/sink/vmemory_scratch_sink.h b/be/src/vec/sink/vmemory_scratch_sink.h index c6306c481a91da6..848952b1d79c4fa 100644 --- a/be/src/vec/sink/vmemory_scratch_sink.h +++ b/be/src/vec/sink/vmemory_scratch_sink.h @@ -60,7 +60,7 @@ class MemoryScratchSink final : public DataSink { Status close(RuntimeState* state, Status exec_status) override; - RuntimeProfile* profile() override { return _profile; } + bool can_write() override; private: Status _prepare_vexpr(RuntimeState* state); diff --git a/be/src/vec/sink/vresult_file_sink.cpp b/be/src/vec/sink/vresult_file_sink.cpp index e30585419bde3ec..66d64140814485c 100644 --- a/be/src/vec/sink/vresult_file_sink.cpp +++ b/be/src/vec/sink/vresult_file_sink.cpp @@ -18,7 +18,6 @@ #include "vec/sink/vresult_file_sink.h" #include -#include #include #include #include @@ -29,14 +28,10 @@ #include "common/config.h" #include "common/object_pool.h" #include "runtime/buffer_control_block.h" -#include "runtime/exec_env.h" #include "runtime/result_buffer_mgr.h" #include "runtime/runtime_state.h" -#include "util/runtime_profile.h" #include "util/telemetry/telemetry.h" -#include "util/uid_util.h" #include "vec/exprs/vexpr.h" -#include "vec/runtime/vfile_result_writer.h" namespace doris { class QueryStatistics; @@ -45,71 +40,45 @@ class TExpr; namespace doris::vectorized { -VResultFileSink::VResultFileSink(RuntimeState* state, ObjectPool* pool, - const RowDescriptor& row_desc, const TResultFileSink& sink, - bool send_query_statistics_with_every_batch, +VResultFileSink::VResultFileSink(const RowDescriptor& row_desc, const std::vector& t_output_expr) - : DataSink(row_desc), _t_output_expr(t_output_expr) { - CHECK(sink.__isset.file_options); - _file_opts.reset(new ResultFileOptions(sink.file_options)); - CHECK(sink.__isset.storage_backend_type); - _storage_type = sink.storage_backend_type; - _is_top_sink = true; - - _name = "VResultFileSink"; - //for impl csv_with_name and csv_with_names_and_types - _header_type = sink.header_type; - _header = sink.header; -} + : AsyncWriterSink(row_desc, t_output_expr) {} VResultFileSink::VResultFileSink(RuntimeState* state, ObjectPool* pool, int sender_id, const RowDescriptor& row_desc, const TResultFileSink& sink, const std::vector& destinations, bool send_query_statistics_with_every_batch, const std::vector& t_output_expr, DescriptorTbl& descs) - : DataSink(row_desc), - _t_output_expr(t_output_expr), + : AsyncWriterSink(row_desc, t_output_expr), _output_row_descriptor(descs.get_tuple_descriptor(sink.output_tuple_id), false) { - CHECK(sink.__isset.file_options); - _file_opts.reset(new ResultFileOptions(sink.file_options)); - CHECK(sink.__isset.storage_backend_type); - _storage_type = sink.storage_backend_type; _is_top_sink = false; CHECK_EQ(destinations.size(), 1); _stream_sender.reset(new VDataStreamSender(state, pool, sender_id, row_desc, sink.dest_node_id, destinations, send_query_statistics_with_every_batch)); - - _name = "VResultFileSink"; - //for impl csv_with_name and csv_with_names_and_types - _header_type = sink.header_type; - _header = sink.header; } Status VResultFileSink::init(const TDataSink& tsink) { if (!_is_top_sink) { RETURN_IF_ERROR(_stream_sender->init(tsink)); } - return Status::OK(); -} -Status VResultFileSink::prepare_exprs(RuntimeState* state) { - // From the thrift expressions create the real exprs. - RETURN_IF_ERROR(VExpr::create_expr_trees(_t_output_expr, _output_vexpr_ctxs)); - // Prepare the exprs to run. - RETURN_IF_ERROR(VExpr::prepare(_output_vexpr_ctxs, state, _row_desc)); - return Status::OK(); + auto& sink = tsink.result_file_sink; + CHECK(sink.__isset.file_options); + _file_opts.reset(new ResultFileOptions(sink.file_options)); + CHECK(sink.__isset.storage_backend_type); + _storage_type = sink.storage_backend_type; + + _name = "VResultFileSink"; + //for impl csv_with_name and csv_with_names_and_types + _header_type = sink.header_type; + _header = sink.header; + + return VExpr::create_expr_trees(_t_output_expr, _output_vexpr_ctxs); } Status VResultFileSink::prepare(RuntimeState* state) { - RETURN_IF_ERROR(DataSink::prepare(state)); - std::stringstream title; - title << "VResultFileSink (fragment_instance_id=" << print_id(state->fragment_instance_id()) - << ")"; - // create profile - _profile = state->obj_pool()->add(new RuntimeProfile(title.str())); - // prepare output_expr - RETURN_IF_ERROR(prepare_exprs(state)); + RETURN_IF_ERROR(AsyncWriterSink::prepare(state)); CHECK(_file_opts.get() != nullptr); if (_is_top_sink) { @@ -120,7 +89,7 @@ Status VResultFileSink::prepare(RuntimeState* state) { // create writer _writer.reset(new (std::nothrow) VFileResultWriter( _file_opts.get(), _storage_type, state->fragment_instance_id(), _output_vexpr_ctxs, - _profile, _sender.get(), nullptr, state->return_object_data_as_binary(), + _sender.get(), nullptr, state->return_object_data_as_binary(), _output_row_descriptor)); } else { // init channel @@ -128,13 +97,12 @@ Status VResultFileSink::prepare(RuntimeState* state) { Block::create_unique(_output_row_descriptor.tuple_descriptors()[0]->slots(), 1); _writer.reset(new (std::nothrow) VFileResultWriter( _file_opts.get(), _storage_type, state->fragment_instance_id(), _output_vexpr_ctxs, - _profile, nullptr, _output_block.get(), state->return_object_data_as_binary(), + nullptr, _output_block.get(), state->return_object_data_as_binary(), _output_row_descriptor)); RETURN_IF_ERROR(_stream_sender->prepare(state)); _profile->add_child(_stream_sender->profile(), true, nullptr); } _writer->set_header_info(_header_type, _header); - RETURN_IF_ERROR(_writer->init(state)); return Status::OK(); } @@ -142,12 +110,7 @@ Status VResultFileSink::open(RuntimeState* state) { if (!_is_top_sink) { RETURN_IF_ERROR(_stream_sender->open(state)); } - return VExpr::open(_output_vexpr_ctxs, state); -} - -Status VResultFileSink::send(RuntimeState* state, Block* block, bool eos) { - RETURN_IF_ERROR(_writer->append_block(*block)); - return Status::OK(); + return AsyncWriterSink::open(state); } Status VResultFileSink::close(RuntimeState* state, Status exec_status) { @@ -157,7 +120,7 @@ Status VResultFileSink::close(RuntimeState* state, Status exec_status) { Status final_status = exec_status; // close the writer - if (_writer) { + if (_writer && _writer->need_normal_close()) { Status st = _writer->close(); if (!st.ok() && exec_status.ok()) { // close file writer failed, should return this error to client diff --git a/be/src/vec/sink/vresult_file_sink.h b/be/src/vec/sink/vresult_file_sink.h index 848a6c371d90b06..fdf4843a528d5b9 100644 --- a/be/src/vec/sink/vresult_file_sink.h +++ b/be/src/vec/sink/vresult_file_sink.h @@ -24,11 +24,12 @@ #include #include "common/status.h" -#include "exec/data_sink.h" #include "runtime/descriptors.h" #include "vec/core/block.h" +#include "vec/sink/async_writer_sink.h" #include "vec/sink/vdata_stream_sender.h" #include "vec/sink/vresult_sink.h" +#include "vec/sink/writer/vfile_result_writer.h" namespace doris { class BufferControlBlock; @@ -44,51 +45,43 @@ class TResultFileSink; namespace vectorized { class VExprContext; -class VResultFileSink : public DataSink { +inline constexpr char VRESULT_FILE_SINK[] = "VResultFileSink"; + +class VResultFileSink : public AsyncWriterSink { public: - VResultFileSink(RuntimeState* state, ObjectPool* pool, const RowDescriptor& row_desc, - const TResultFileSink& sink, bool send_query_statistics_with_every_batch, - const std::vector& t_output_expr); + VResultFileSink(const RowDescriptor& row_desc, const std::vector& t_output_expr); + VResultFileSink(RuntimeState* state, ObjectPool* pool, int sender_id, const RowDescriptor& row_desc, const TResultFileSink& sink, const std::vector& destinations, bool send_query_statistics_with_every_batch, const std::vector& t_output_expr, DescriptorTbl& descs); - ~VResultFileSink() override = default; + Status init(const TDataSink& thrift_sink) override; Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; - // send data in 'batch' to this backend stream mgr - // Blocks until all rows in batch are placed in the buffer - Status send(RuntimeState* state, Block* block, bool eos = false) override; + // Flush all buffered data and close all existing channels to destination // hosts. Further send() calls are illegal after calling close(). Status close(RuntimeState* state, Status exec_status) override; - RuntimeProfile* profile() override { return _profile; } void set_query_statistics(std::shared_ptr statistics) override; private: - Status prepare_exprs(RuntimeState* state); // set file options when sink type is FILE std::unique_ptr _file_opts; TStorageBackendType::type _storage_type; // Owned by the RuntimeState. - const std::vector& _t_output_expr; - VExprContextSPtrs _output_vexpr_ctxs; RowDescriptor _output_row_descriptor; std::unique_ptr _output_block = nullptr; std::shared_ptr _sender; std::unique_ptr _stream_sender; - std::shared_ptr _writer; int _buf_size = 1024; // Allocated from _pool bool _is_top_sink = true; std::string _header; std::string _header_type; - - RuntimeProfile* _profile; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/sink/vresult_sink.cpp b/be/src/vec/sink/vresult_sink.cpp index 2e82011bd7b11a6..5fdaaf162826c3a 100644 --- a/be/src/vec/sink/vresult_sink.cpp +++ b/be/src/vec/sink/vresult_sink.cpp @@ -38,6 +38,7 @@ #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" #include "vec/sink/vmysql_result_writer.h" +#include "vec/sink/writer/vfile_result_writer.h" namespace doris { class QueryStatistics; diff --git a/be/src/vec/sink/vresult_sink.h b/be/src/vec/sink/vresult_sink.h index c9374d7cf93ca22..a10e60d4672a5f6 100644 --- a/be/src/vec/sink/vresult_sink.h +++ b/be/src/vec/sink/vresult_sink.h @@ -135,7 +135,6 @@ class VResultSink : public DataSink { // Flush all buffered data and close all existing channels to destination // hosts. Further send() calls are illegal after calling close(). virtual Status close(RuntimeState* state, Status exec_status) override; - virtual RuntimeProfile* profile() override { return _profile; } void set_query_statistics(std::shared_ptr statistics) override; @@ -152,8 +151,7 @@ class VResultSink : public DataSink { std::shared_ptr _sender; std::shared_ptr _writer; - RuntimeProfile* _profile; // Allocated from _pool - int _buf_size; // Allocated from _pool + int _buf_size; // Allocated from _pool // for fetch data by rowids TFetchOption _fetch_option; diff --git a/be/src/vec/sink/vtablet_block_convertor.cpp b/be/src/vec/sink/vtablet_block_convertor.cpp index 7c7300b9b382581..45105e0cfffe4ef 100644 --- a/be/src/vec/sink/vtablet_block_convertor.cpp +++ b/be/src/vec/sink/vtablet_block_convertor.cpp @@ -131,6 +131,33 @@ DecimalV2Value OlapTableBlockConvertor::_get_decimalv2_min_or_max(const TypeDesc return value; } +template +DecimalType OlapTableBlockConvertor::_get_decimalv3_min_or_max(const TypeDescriptor& type) { + std::map* pmap; + if constexpr (std::is_same_v) { + pmap = IsMin ? &_min_decimal32_val : &_max_decimal32_val; + } else if constexpr (std::is_same_v) { + pmap = IsMin ? &_min_decimal64_val : &_max_decimal64_val; + } else { + pmap = IsMin ? &_min_decimal128_val : &_max_decimal128_val; + } + + // found + auto iter = pmap->find(type.precision); + if (iter != pmap->end()) { + return iter->second; + } + + typename DecimalType::NativeType value; + if constexpr (IsMin) { + value = vectorized::min_decimal_value(type.precision); + } else { + value = vectorized::max_decimal_value(type.precision); + } + pmap->emplace(type.precision, value); + return value; +} + Status OlapTableBlockConvertor::_validate_column(RuntimeState* state, const TypeDescriptor& type, bool is_nullable, vectorized::ColumnPtr column, size_t slot_index, bool* stop_processing, @@ -269,8 +296,8 @@ Status OlapTableBlockConvertor::_validate_column(RuntimeState* state, const Type #define CHECK_VALIDATION_FOR_DECIMALV3(DecimalType) \ auto column_decimal = const_cast*>( \ assert_cast*>(real_column_ptr.get())); \ - const auto& max_decimal = type_limit::max(); \ - const auto& min_decimal = type_limit::min(); \ + const auto& max_decimal = _get_decimalv3_min_or_max(type); \ + const auto& min_decimal = _get_decimalv3_min_or_max(type); \ for (size_t j = 0; j < column->size(); ++j) { \ auto row = rows ? (*rows)[j] : j; \ if (row == last_invalid_row) { \ diff --git a/be/src/vec/sink/vtablet_block_convertor.h b/be/src/vec/sink/vtablet_block_convertor.h index 335e876284af8ef..bfc7b3b5d969dee 100644 --- a/be/src/vec/sink/vtablet_block_convertor.h +++ b/be/src/vec/sink/vtablet_block_convertor.h @@ -62,6 +62,9 @@ class OlapTableBlockConvertor { template DecimalV2Value _get_decimalv2_min_or_max(const TypeDescriptor& type); + template + DecimalType _get_decimalv3_min_or_max(const TypeDescriptor& type); + Status _validate_column(RuntimeState* state, const TypeDescriptor& type, bool is_nullable, vectorized::ColumnPtr column, size_t slot_index, bool* stop_processing, fmt::memory_buffer& error_prefix, diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp index 0b6a2e7a32f429e..02782a822074ba3 100644 --- a/be/src/vec/sink/vtablet_sink.cpp +++ b/be/src/vec/sink/vtablet_sink.cpp @@ -730,7 +730,7 @@ Status VNodeChannel::none_of(std::initializer_list vars) { if (!vars_str.empty()) { vars_str.pop_back(); // 0/1/0/ -> 0/1/0 } - st = Status::InternalError(vars_str); + st = Status::Uninitialized(vars_str); } return st; @@ -757,6 +757,8 @@ void VNodeChannel::try_send_block(RuntimeState* state) { // tablet_ids has already set when add row request.set_packet_seq(_next_packet_seq); auto block = mutable_block->to_block(); + CHECK(block.rows() == request.tablet_ids_size()) + << "block rows: " << block.rows() << ", tablet_ids_size: " << request.tablet_ids_size(); if (block.rows() > 0) { SCOPED_ATOMIC_TIMER(&_serialize_batch_ns); size_t uncompressed_bytes = 0, compressed_bytes = 0; diff --git a/be/src/vec/sink/vtablet_sink.h b/be/src/vec/sink/vtablet_sink.h index e6d4992b4831440..dc408d392b24c97 100644 --- a/be/src/vec/sink/vtablet_sink.h +++ b/be/src/vec/sink/vtablet_sink.h @@ -408,9 +408,6 @@ class VOlapTableSink final : public DataSink { size_t get_pending_bytes() const; - // Returns the runtime profile for the sink. - RuntimeProfile* profile() override { return _profile; } - // the consumer func of sending pending batches in every NodeChannel. // use polling & NodeChannel::try_send_and_fetch_status() to achieve nonblocking sending. // only focus on pending batches and channel status, the internal errors of NodeChannels will be handled by the producer @@ -464,8 +461,6 @@ class VOlapTableSink final : public DataSink { OlapTableLocationParam* _slave_location = nullptr; DorisNodesInfo* _nodes_info = nullptr; - RuntimeProfile* _profile = nullptr; - std::unique_ptr _tablet_finder; // index_channel diff --git a/be/src/vec/sink/vtablet_sink_v2.cpp b/be/src/vec/sink/vtablet_sink_v2.cpp index b77e44394f998a5..eb66eac664c7517 100644 --- a/be/src/vec/sink/vtablet_sink_v2.cpp +++ b/be/src/vec/sink/vtablet_sink_v2.cpp @@ -39,7 +39,6 @@ #include "common/object_pool.h" #include "common/status.h" #include "exec/tablet_info.h" -#include "io/fs/stream_sink_file_writer.h" #include "olap/delta_writer_v2.h" #include "runtime/descriptors.h" #include "runtime/exec_env.h" @@ -55,6 +54,7 @@ #include "util/uid_util.h" #include "vec/core/block.h" #include "vec/exprs/vexpr.h" +#include "vec/sink/load_stream_stub.h" #include "vec/sink/vtablet_block_convertor.h" #include "vec/sink/vtablet_finder.h" @@ -63,76 +63,6 @@ class TExpr; namespace stream_load { -int StreamSinkHandler::on_received_messages(brpc::StreamId id, butil::IOBuf* const messages[], - size_t size) { - int64_t backend_id = _sink->_node_id_for_stream->at(id); - - for (size_t i = 0; i < size; i++) { - butil::IOBufAsZeroCopyInputStream wrapper(*messages[i]); - PWriteStreamSinkResponse response; - response.ParseFromZeroCopyStream(&wrapper); - - Status st = Status::create(response.status()); - - std::stringstream ss; - ss << "received response from backend " << backend_id << ", status: " << st - << ", success tablet ids:"; - for (auto tablet_id : response.success_tablet_ids()) { - ss << " " << tablet_id; - } - ss << ", failed tablet ids:"; - for (auto tablet_id : response.failed_tablet_ids()) { - ss << " " << tablet_id; - } - LOG(INFO) << ss.str(); - - int replica = _sink->_num_replicas; - - { - std::lock_guard l(_sink->_tablet_success_map_mutex); - for (auto tablet_id : response.success_tablet_ids()) { - if (_sink->_tablet_success_map.count(tablet_id) == 0) { - _sink->_tablet_success_map.insert({tablet_id, {}}); - } - _sink->_tablet_success_map[tablet_id].push_back(backend_id); - } - } - { - std::lock_guard l(_sink->_tablet_failure_map_mutex); - for (auto tablet_id : response.failed_tablet_ids()) { - if (_sink->_tablet_failure_map.count(tablet_id) == 0) { - _sink->_tablet_failure_map.insert({tablet_id, {}}); - } - _sink->_tablet_failure_map[tablet_id].push_back(backend_id); - if (_sink->_tablet_failure_map[tablet_id].size() * 2 >= replica) { - _sink->_cancel(Status::Cancelled( - "Failed to meet num replicas requirements for tablet {}", tablet_id)); - break; - } - } - } - - if (response.has_load_stream_profile()) { - TRuntimeProfileTree tprofile; - const uint8_t* buf = - reinterpret_cast(response.load_stream_profile().data()); - uint32_t len = response.load_stream_profile().size(); - auto status = deserialize_thrift_msg(buf, &len, false, &tprofile); - if (status.ok()) { - _sink->_state->load_channel_profile()->update(tprofile); - } else { - LOG(WARNING) << "load channel TRuntimeProfileTree deserialize failed, errmsg=" - << status; - } - } - - _sink->_pending_reports.fetch_add(-1); - } - return 0; -} - -void StreamSinkHandler::on_closed(brpc::StreamId id) {} - VOlapTableSinkV2::VOlapTableSinkV2(ObjectPool* pool, const RowDescriptor& row_desc, const std::vector& texprs, Status* status) : DataSink(row_desc), _pool(pool) { @@ -210,7 +140,6 @@ Status VOlapTableSinkV2::prepare(RuntimeState* state) { _close_timer = ADD_TIMER(_profile, "CloseWaitTime"); _close_writer_timer = ADD_CHILD_TIMER(_profile, "CloseWriterTime", "CloseWaitTime"); _close_load_timer = ADD_CHILD_TIMER(_profile, "CloseLoadTime", "CloseWaitTime"); - _close_stream_timer = ADD_CHILD_TIMER(_profile, "CloseStreamTime", "CloseWaitTime"); // Prepare the exprs to run. RETURN_IF_ERROR(vectorized::VExpr::prepare(_output_vexpr_ctxs, state, _row_desc)); @@ -224,8 +153,7 @@ Status VOlapTableSinkV2::open(RuntimeState* state) { SCOPED_TIMER(_open_timer); SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); - _stream_pool_for_node = std::make_shared(); - _node_id_for_stream = std::make_shared(); + _stream_pool_for_node = std::make_shared(); _delta_writer_for_tablet = std::make_shared(); _build_tablet_node_mapping(); RETURN_IF_ERROR(_init_stream_pools()); @@ -234,70 +162,32 @@ Status VOlapTableSinkV2::open(RuntimeState* state) { } Status VOlapTableSinkV2::_init_stream_pools() { + // stub template is for sharing internal schema map among all stubs + LoadStreamStub stub_template {_load_id, _sender_id}; for (auto& [node_id, _] : _tablets_for_node) { auto node_info = _nodes_info->find_node(node_id); if (node_info == nullptr) { return Status::InternalError("Unknown node {} in tablet location", node_id); } - _stream_pool_for_node->insert({node_id, StreamPool {}}); - StreamPool& stream_pool = _stream_pool_for_node->at(node_id); - RETURN_IF_ERROR(_init_stream_pool(*node_info, stream_pool)); - for (auto stream : stream_pool) { - _node_id_for_stream->insert({stream, node_id}); - } + Streams& stream_pool = (*_stream_pool_for_node)[node_id]; + RETURN_IF_ERROR(_init_stream_pool(*node_info, stream_pool, stub_template)); } return Status::OK(); } -Status VOlapTableSinkV2::_init_stream_pool(const NodeInfo& node_info, StreamPool& stream_pool) { - DCHECK_GT(config::num_streams_per_sink, 0); +Status VOlapTableSinkV2::_init_stream_pool(const NodeInfo& node_info, Streams& stream_pool, + LoadStreamStub& stub_template) { stream_pool.reserve(config::num_streams_per_sink); for (int i = 0; i < config::num_streams_per_sink; ++i) { - brpc::StreamOptions opt; - opt.max_buf_size = 20 << 20; // 20MB - opt.idle_timeout_ms = 30000; - opt.messages_in_batch = 128; - opt.handler = new StreamSinkHandler(this); - brpc::StreamId stream; - brpc::Controller cntl; - if (int ret = StreamCreate(&stream, cntl, &opt)) { - return Status::RpcError("Failed to create stream, code = {}", ret); - } - LOG(INFO) << "Created stream " << stream << " for backend " << node_info.id << " (" - << node_info.host << ":" << node_info.brpc_port << ")"; - std::string host_port = get_host_port(node_info.host, node_info.brpc_port); - // use "pooled" connection to avoid conflicts between streaming rpc and regular rpc, - // see: https://github.com/apache/brpc/issues/392 - const auto& stub = - _state->exec_env()->brpc_internal_client_cache()->get_new_client_no_cache( - host_port, "baidu_std", "pooled"); - POpenStreamSinkRequest request; - *request.mutable_load_id() = _load_id; - request.set_src_id(_sender_id); - request.set_txn_id(_txn_id); - request.set_enable_profile(_state->enable_profile()); - _schema->to_protobuf(request.mutable_schema()); - if (i == 0) { - // get tablet schema from each backend only in the 1st stream - for (auto& tablet : _indexes_from_node[node_info.id]) { - auto req = request.add_tablets(); - *req = tablet; - } - } - POpenStreamSinkResponse response; - cntl.set_timeout_ms(config::open_stream_sink_timeout_ms); - stub->open_stream_sink(&cntl, &request, &response, nullptr); - for (const auto& resp : response.tablet_schemas()) { - auto tablet_schema = std::make_shared(); - tablet_schema->init_from_pb(resp.tablet_schema()); - _tablet_schema_for_index[resp.index_id()] = tablet_schema; - _enable_unique_mow_for_index[resp.index_id()] = resp.enable_unique_key_merge_on_write(); - } - if (cntl.Failed()) { - return Status::InternalError("Failed to connect to backend {}: {}", node_info.id, - cntl.ErrorText()); - } - stream_pool.push_back(stream); + // internal tablet schema map will be shared among all stubs + auto stream = std::make_unique(stub_template); + // get tablet schema from each backend only in the 1st stream + const std::vector& tablets_for_schema = + i == 0 ? _indexes_from_node[node_info.id] : std::vector {}; + RETURN_IF_ERROR(stream->open(_state->exec_env()->brpc_internal_client_cache(), node_info, + _txn_id, *_schema, tablets_for_schema, + _state->enable_profile())); + stream_pool.emplace_back(std::move(stream)); } return Status::OK(); } @@ -342,13 +232,13 @@ void VOlapTableSinkV2::_generate_rows_for_tablet(RowsForTablet& rows_for_tablet, } } -Status VOlapTableSinkV2::_select_streams(int64_t tablet_id, std::vector& streams) { +Status VOlapTableSinkV2::_select_streams(int64_t tablet_id, Streams& streams) { auto location = _location->find_tablet(tablet_id); if (location == nullptr) { return Status::InternalError("unknown tablet location, tablet id = {}", tablet_id); } for (auto& node_id : location->node_ids) { - streams.push_back(_stream_pool_for_node->at(node_id)[_stream_index]); + streams.emplace_back(_stream_pool_for_node->at(node_id)[_stream_index]); } _stream_index = (_stream_index + 1) % config::num_streams_per_sink; return Status::OK(); @@ -358,8 +248,6 @@ Status VOlapTableSinkV2::send(RuntimeState* state, vectorized::Block* input_bloc SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); Status status = Status::OK(); - LOG(INFO) << "upstream id = " << state->backend_id(); - auto input_rows = input_block->rows(); auto input_bytes = input_block->bytes(); if (UNLIKELY(input_rows == 0)) { @@ -407,7 +295,7 @@ Status VOlapTableSinkV2::send(RuntimeState* state, vectorized::Block* input_bloc // For each tablet, send its input_rows from block to delta writer for (const auto& [tablet_id, rows] : rows_for_tablet) { - std::vector streams; + Streams streams; RETURN_IF_ERROR(_select_streams(tablet_id, streams)); RETURN_IF_ERROR(_write_memtable(block, tablet_id, rows, streams)); } @@ -417,7 +305,7 @@ Status VOlapTableSinkV2::send(RuntimeState* state, vectorized::Block* input_bloc Status VOlapTableSinkV2::_write_memtable(std::shared_ptr block, int64_t tablet_id, const Rows& rows, - const std::vector& streams) { + const Streams& streams) { DeltaWriterV2* delta_writer = nullptr; { auto it = _delta_writer_for_tablet->find(tablet_id); @@ -433,10 +321,6 @@ Status VOlapTableSinkV2::_write_memtable(std::shared_ptr bloc req.tuple_desc = _output_tuple_desc; req.is_high_priority = _is_high_priority; req.table_schema_param = _schema.get(); - req.tablet_schema = _tablet_schema_for_index[rows.index_id]; - req.enable_unique_key_merge_on_write = _enable_unique_mow_for_index[rows.index_id]; - req.sender_id = _sender_id; - req.streams = streams; for (auto& index : _schema->indexes()) { if (index->index_id == rows.index_id) { req.slots = &index->slots; @@ -444,9 +328,8 @@ Status VOlapTableSinkV2::_write_memtable(std::shared_ptr bloc break; } } - DeltaWriterV2::open(&req, &delta_writer, _profile); - _delta_writer_for_tablet->insert( - {tablet_id, std::unique_ptr(delta_writer)}); + DeltaWriterV2::open(&req, streams, &delta_writer, _profile); + _delta_writer_for_tablet->emplace(tablet_id, delta_writer); } else { VLOG_DEBUG << "Reusing DeltaWriterV2 for Tablet(tablet id: " << tablet_id << ", index id: " << rows.index_id << ")"; @@ -471,11 +354,6 @@ Status VOlapTableSinkV2::_cancel(Status status) { [&status](auto&& entry) { entry.second->cancel_with_status(status); }); } _delta_writer_for_tablet.reset(); - if (_stream_pool_for_node.use_count() == 1) { - std::for_each(std::begin(*_node_id_for_stream), std::end(*_node_id_for_stream), - [](auto&& entry) { brpc::StreamClose(entry.first); }); - } - _stream_pool_for_node.reset(); return Status::OK(); } @@ -514,43 +392,35 @@ Status VOlapTableSinkV2::close(RuntimeState* state, Status exec_status) { { // send CLOSE_LOAD to all streams, return ERROR if any - RETURN_IF_ERROR(std::transform_reduce( - std::begin(*_node_id_for_stream), std::end(*_node_id_for_stream), Status::OK(), - [](Status& left, Status&& right) { return left.ok() ? right : left; }, - [this](auto&& entry) { return _close_load(entry.first); })); - } - - { - SCOPED_TIMER(_close_load_timer); - while (_pending_reports.load() > 0) { - // TODO: use a better wait - std::this_thread::sleep_for(std::chrono::milliseconds(1)); - LOG(INFO) << "sinkv2 close_wait, pending reports: " << _pending_reports.load(); + for (const auto& [node_id, stream_pool] : *_stream_pool_for_node) { + RETURN_IF_ERROR(_close_load(stream_pool)); } } { - SCOPED_TIMER(_close_stream_timer); - // close streams - if (_stream_pool_for_node.use_count() == 1) { - std::for_each(std::begin(*_node_id_for_stream), std::end(*_node_id_for_stream), - [](auto&& entry) { brpc::StreamClose(entry.first); }); + SCOPED_TIMER(_close_load_timer); + for (const auto& [node_id, stream_pool] : *_stream_pool_for_node) { + for (const auto& stream : stream_pool) { + stream->close_wait(); + } } - _stream_pool_for_node.reset(); } std::vector tablet_commit_infos; - for (auto& [tablet_id, backends] : _tablet_success_map) { - for (int64_t be_id : backends) { - TTabletCommitInfo commit_info; - commit_info.tabletId = tablet_id; - commit_info.backendId = be_id; - tablet_commit_infos.emplace_back(std::move(commit_info)); + for (const auto& [node_id, stream_pool] : *_stream_pool_for_node) { + for (const auto& stream : stream_pool) { + for (auto tablet_id : stream->success_tablets()) { + TTabletCommitInfo commit_info; + commit_info.tabletId = tablet_id; + commit_info.backendId = node_id; + tablet_commit_infos.emplace_back(std::move(commit_info)); + } } } state->tablet_commit_infos().insert(state->tablet_commit_infos().end(), std::make_move_iterator(tablet_commit_infos.begin()), std::make_move_iterator(tablet_commit_infos.end())); + _stream_pool_for_node.reset(); // _number_input_rows don't contain num_rows_load_filtered and num_rows_load_unselected in scan node int64_t num_rows_load_total = _number_input_rows + state->num_rows_load_filtered() + @@ -572,28 +442,17 @@ Status VOlapTableSinkV2::close(RuntimeState* state, Status exec_status) { return status; } -Status VOlapTableSinkV2::_close_load(brpc::StreamId stream) { - butil::IOBuf buf; - PStreamHeader header; - *header.mutable_load_id() = _load_id; - header.set_src_id(_sender_id); - header.set_opcode(doris::PStreamHeader::CLOSE_LOAD); - auto node_id = _node_id_for_stream.get()->at(stream); +Status VOlapTableSinkV2::_close_load(const Streams& streams) { + auto node_id = streams[0]->dst_id(); + std::vector tablets_to_commit; for (auto tablet : _tablets_for_node[node_id]) { - int64_t partition_id = tablet.partition_id(); - if (_tablet_finder->partition_ids().contains(tablet.partition_id()) || - _send_partitions_recorder[node_id].find(partition_id) == - _send_partitions_recorder[node_id].end()) { - PTabletID* tablet_to_commit = header.add_tablets_to_commit(); - *tablet_to_commit = tablet; - _send_partitions_recorder[node_id].insert(tablet.partition_id()); + if (_tablet_finder->partition_ids().contains(tablet.partition_id())) { + tablets_to_commit.push_back(tablet); } } - size_t header_len = header.ByteSizeLong(); - buf.append(reinterpret_cast(&header_len), sizeof(header_len)); - buf.append(header.SerializeAsString()); - _pending_reports.fetch_add(1); - io::StreamSinkFileWriter::send_with_retry(stream, buf); + for (const auto& stream : streams) { + RETURN_IF_ERROR(stream->close_load(tablets_to_commit)); + } return Status::OK(); } diff --git a/be/src/vec/sink/vtablet_sink_v2.h b/be/src/vec/sink/vtablet_sink_v2.h index 6c758d643c4bdf7..047377f4e28d24c 100644 --- a/be/src/vec/sink/vtablet_sink_v2.h +++ b/be/src/vec/sink/vtablet_sink_v2.h @@ -66,6 +66,7 @@ namespace doris { class DeltaWriterV2; +class LoadStreamStub; class ObjectPool; class RowDescriptor; class RuntimeState; @@ -81,8 +82,8 @@ class OlapTabletFinder; class VOlapTableSinkV2; using DeltaWriterForTablet = std::unordered_map>; -using StreamPool = std::vector; -using StreamPoolForNode = std::unordered_map; +using Streams = std::vector>; +using NodeToStreams = std::unordered_map; using NodeIdForStream = std::unordered_map; using NodePartitionTabletMapping = std::unordered_map>>; @@ -131,11 +132,9 @@ class VOlapTableSinkV2 final : public DataSink { Status close(RuntimeState* state, Status close_status) override; Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override; - // Returns the runtime profile for the sink. - RuntimeProfile* profile() override { return _profile; } - private: - Status _init_stream_pool(const NodeInfo& node_info, StreamPool& stream_pool); + Status _init_stream_pool(const NodeInfo& node_info, Streams& stream_pool, + LoadStreamStub& stub_template); Status _init_stream_pools(); @@ -146,11 +145,11 @@ class VOlapTableSinkV2 final : public DataSink { int row_idx); Status _write_memtable(std::shared_ptr block, int64_t tablet_id, - const Rows& rows, const std::vector& streams); + const Rows& rows, const Streams& streams); - Status _select_streams(int64_t tablet_id, std::vector& streams); + Status _select_streams(int64_t tablet_id, Streams& streams); - Status _close_load(brpc::StreamId stream); + Status _close_load(const Streams& streams); Status _cancel(Status status); @@ -176,13 +175,9 @@ class VOlapTableSinkV2 final : public DataSink { // TODO(zc): think about cache this data std::shared_ptr _schema; - std::unordered_map> _tablet_schema_for_index; - std::unordered_map _enable_unique_mow_for_index; OlapTableLocationParam* _location = nullptr; DorisNodesInfo* _nodes_info = nullptr; - RuntimeProfile* _profile = nullptr; - std::unique_ptr _tablet_finder; std::unique_ptr _block_convertor; @@ -206,7 +201,6 @@ class VOlapTableSinkV2 final : public DataSink { RuntimeProfile::Counter* _close_timer = nullptr; RuntimeProfile::Counter* _close_writer_timer = nullptr; RuntimeProfile::Counter* _close_load_timer = nullptr; - RuntimeProfile::Counter* _close_stream_timer = nullptr; // Save the status of close() method Status _close_status; @@ -220,14 +214,12 @@ class VOlapTableSinkV2 final : public DataSink { std::unordered_map> _tablets_for_node; std::unordered_map> _indexes_from_node; - std::unordered_map> _send_partitions_recorder; - std::shared_ptr _stream_pool_for_node; - std::shared_ptr _node_id_for_stream; + std::shared_ptr _stream_pool_for_node; size_t _stream_index = 0; std::shared_ptr _delta_writer_for_tablet; - std::atomic _pending_reports {0}; + std::atomic _pending_streams {0}; std::unordered_map> _tablet_success_map; std::unordered_map> _tablet_failure_map; diff --git a/be/src/vec/sink/writer/async_result_writer.h b/be/src/vec/sink/writer/async_result_writer.h index b294e110a964d15..2f70b05266b74f9 100644 --- a/be/src/vec/sink/writer/async_result_writer.h +++ b/be/src/vec/sink/writer/async_result_writer.h @@ -54,7 +54,9 @@ class AsyncResultWriter : public ResultWriter { virtual bool in_transaction() { return false; } - bool need_normal_close() { return _need_normal_close; } + virtual Status commit_trans() { return Status::OK(); } + + bool need_normal_close() const { return _need_normal_close; } Status init(RuntimeState* state) override { return Status::OK(); } diff --git a/be/src/vec/runtime/vfile_result_writer.cpp b/be/src/vec/sink/writer/vfile_result_writer.cpp similarity index 94% rename from be/src/vec/runtime/vfile_result_writer.cpp rename to be/src/vec/sink/writer/vfile_result_writer.cpp index 9d5fc4e1588756a..e54c426f74ff159 100644 --- a/be/src/vec/runtime/vfile_result_writer.cpp +++ b/be/src/vec/sink/writer/vfile_result_writer.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "vec/runtime/vfile_result_writer.h" +#include "vfile_result_writer.h" #include #include @@ -72,18 +72,20 @@ namespace doris::vectorized { const size_t VFileResultWriter::OUTSTREAM_BUFFER_SIZE_BYTES = 1024 * 1024; using doris::operator<<; +VFileResultWriter::VFileResultWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs) + : AsyncResultWriter(output_exprs) {} + VFileResultWriter::VFileResultWriter(const ResultFileOptions* file_opts, const TStorageBackendType::type storage_type, const TUniqueId fragment_instance_id, const VExprContextSPtrs& output_vexpr_ctxs, - RuntimeProfile* parent_profile, BufferControlBlock* sinker, - Block* output_block, bool output_object_data, + BufferControlBlock* sinker, Block* output_block, + bool output_object_data, const RowDescriptor& output_row_descriptor) - : _file_opts(file_opts), + : AsyncResultWriter(output_vexpr_ctxs), + _file_opts(file_opts), _storage_type(storage_type), _fragment_instance_id(fragment_instance_id), - _output_vexpr_ctxs(output_vexpr_ctxs), - _parent_profile(parent_profile), _sinker(sinker), _output_block(output_block), _output_row_descriptor(output_row_descriptor), @@ -91,9 +93,9 @@ VFileResultWriter::VFileResultWriter(const ResultFileOptions* file_opts, _output_object_data = output_object_data; } -Status VFileResultWriter::init(RuntimeState* state) { +Status VFileResultWriter::_init(RuntimeState* state, RuntimeProfile* profile) { _state = state; - _init_profile(); + _init_profile(profile); // Delete existing files if (_file_opts->delete_existing_files) { RETURN_IF_ERROR(_delete_dir()); @@ -101,8 +103,8 @@ Status VFileResultWriter::init(RuntimeState* state) { return _create_next_file_writer(); } -void VFileResultWriter::_init_profile() { - RuntimeProfile* profile = _parent_profile->create_child("VFileResultWriter", true, true); +void VFileResultWriter::_init_profile(RuntimeProfile* parent_profile) { + RuntimeProfile* profile = parent_profile->create_child("VFileResultWriter", true, true); _append_row_batch_timer = ADD_TIMER(profile, "AppendBatchTime"); _convert_tuple_timer = ADD_CHILD_TIMER(profile, "TupleConvertTime", "AppendBatchTime"); _file_write_timer = ADD_CHILD_TIMER(profile, "FileWriteTime", "AppendBatchTime"); @@ -160,13 +162,13 @@ Status VFileResultWriter::_create_file_writer(const std::string& file_name) { break; case TFileFormatType::FORMAT_PARQUET: _vfile_writer.reset(new VParquetWriterWrapper( - _file_writer_impl.get(), _output_vexpr_ctxs, _file_opts->parquet_schemas, + _file_writer_impl.get(), _vec_output_expr_ctxs, _file_opts->parquet_schemas, _file_opts->parquet_commpression_type, _file_opts->parquert_disable_dictionary, _file_opts->parquet_version, _output_object_data)); RETURN_IF_ERROR(_vfile_writer->prepare()); break; case TFileFormatType::FORMAT_ORC: - _vfile_writer.reset(new VOrcWriterWrapper(_file_writer_impl.get(), _output_vexpr_ctxs, + _vfile_writer.reset(new VOrcWriterWrapper(_file_writer_impl.get(), _vec_output_expr_ctxs, _file_opts->orc_schema, _output_object_data)); RETURN_IF_ERROR(_vfile_writer->prepare()); break; @@ -237,12 +239,9 @@ Status VFileResultWriter::append_block(Block& block) { } RETURN_IF_ERROR(write_csv_header()); SCOPED_TIMER(_append_row_batch_timer); - Status status = Status::OK(); - // Exec vectorized expr here to speed up, block.rows() == 0 means expr exec - // failed, just return the error status Block output_block; - RETURN_IF_ERROR(VExprContext::get_output_block_after_execute_exprs(_output_vexpr_ctxs, block, - &output_block)); + RETURN_IF_ERROR(_projection_block(block, &output_block)); + if (_vfile_writer) { RETURN_IF_ERROR(_write_file(output_block)); } else { @@ -267,7 +266,7 @@ Status VFileResultWriter::_write_csv_file(const Block& block) { if (col.column->is_null_at(i)) { _plain_text_outstream << NULL_IN_CSV; } else { - switch (_output_vexpr_ctxs[col_id]->root()->type().type) { + switch (_vec_output_expr_ctxs[col_id]->root()->type().type) { case TYPE_BOOLEAN: case TYPE_TINYINT: _plain_text_outstream << (int)*reinterpret_cast( @@ -327,7 +326,7 @@ Status VFileResultWriter::_write_csv_file(const Block& block) { const DateV2Value* time_val = (const DateV2Value*)(col.column->get_data_at(i) .data); - time_val->to_string(buf, _output_vexpr_ctxs[col_id]->root()->type().scale); + time_val->to_string(buf, _vec_output_expr_ctxs[col_id]->root()->type().scale); _plain_text_outstream << buf; break; } @@ -406,9 +405,9 @@ Status VFileResultWriter::_write_csv_file(const Block& block) { std::string VFileResultWriter::gen_types() { std::string types; - int num_columns = _output_vexpr_ctxs.size(); + int num_columns = _vec_output_expr_ctxs.size(); for (int i = 0; i < num_columns; ++i) { - types += type_to_string(_output_vexpr_ctxs[i]->root()->type().type); + types += type_to_string(_vec_output_expr_ctxs[i]->root()->type().type); if (i < num_columns - 1) { types += _file_opts->column_separator; } @@ -419,7 +418,7 @@ std::string VFileResultWriter::gen_types() { Status VFileResultWriter::write_csv_header() { if (!_header_sent && _header.size() > 0) { - std::string tmp_header = _header; + std::string tmp_header(_header); if (_header_type == BeConsts::CSV_WITH_NAMES_AND_TYPES) { tmp_header += gen_types(); } @@ -628,9 +627,13 @@ Status VFileResultWriter::close() { // because `_close_file_writer()` may be called in deconstructor, // at that time, the RuntimeState may already been deconstructed, // so does the profile in RuntimeState. - COUNTER_SET(_written_rows_counter, _written_rows); - SCOPED_TIMER(_writer_close_timer); + if (_written_rows_counter) { + COUNTER_SET(_written_rows_counter, _written_rows); + SCOPED_TIMER(_writer_close_timer); + } return _close_file_writer(true); } +const string VFileResultWriter::NULL_IN_CSV = "\\N"; + } // namespace doris::vectorized diff --git a/be/src/vec/runtime/vfile_result_writer.h b/be/src/vec/sink/writer/vfile_result_writer.h similarity index 85% rename from be/src/vec/runtime/vfile_result_writer.h rename to be/src/vec/sink/writer/vfile_result_writer.h index 4ec425755d3a311..b56e41c377b08e1 100644 --- a/be/src/vec/runtime/vfile_result_writer.h +++ b/be/src/vec/sink/writer/vfile_result_writer.h @@ -29,10 +29,10 @@ #include "common/status.h" #include "io/fs/file_writer.h" #include "runtime/descriptors.h" -#include "runtime/result_writer.h" #include "util/runtime_profile.h" #include "vec/core/block.h" -#include "vec/runtime/vparquet_writer.h" +#include "vec/runtime/vfile_writer_wrapper.h" +#include "vec/sink/writer/async_result_writer.h" namespace doris { class BufferControlBlock; @@ -47,35 +47,45 @@ struct ResultFileOptions; namespace doris::vectorized { // write result to file -class VFileResultWriter final : public ResultWriter { +class VFileResultWriter final : public AsyncResultWriter { public: VFileResultWriter(const ResultFileOptions* file_option, const TStorageBackendType::type storage_type, const TUniqueId fragment_instance_id, - const VExprContextSPtrs& _output_vexpr_ctxs, RuntimeProfile* parent_profile, - BufferControlBlock* sinker, Block* output_block, bool output_object_data, + const VExprContextSPtrs& _output_vexpr_ctxs, BufferControlBlock* sinker, + Block* output_block, bool output_object_data, const RowDescriptor& output_row_descriptor); - virtual ~VFileResultWriter() = default; + + VFileResultWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs); Status append_block(Block& block) override; - Status init(RuntimeState* state) override; Status close() override; + Status open(RuntimeState* state, RuntimeProfile* profile) override { + return _init(state, profile); + } + // file result writer always return statistic result in one row int64_t get_written_rows() const override { return 1; } std::string gen_types(); Status write_csv_header(); + void set_header_info(const std::string& header_type, const std::string& header) { + _header_type = header_type; + _header = header; + } + private: + Status _init(RuntimeState* state, RuntimeProfile*); Status _write_file(const Block& block); Status _write_csv_file(const Block& block); // if buffer exceed the limit, write the data buffered in _plain_text_outstream via file_writer // if eos, write the data even if buffer is not full. Status _flush_plain_text_outstream(bool eos); - void _init_profile(); + void _init_profile(RuntimeProfile*); Status _create_file_writer(const std::string& file_name); Status _create_next_file_writer(); @@ -96,11 +106,12 @@ class VFileResultWriter final : public ResultWriter { // delete the dir of file_path Status _delete_dir(); + static const std::string NULL_IN_CSV; + RuntimeState* _state; // not owned, set when init const ResultFileOptions* _file_opts; TStorageBackendType::type _storage_type; TUniqueId _fragment_instance_id; - const VExprContextSPtrs& _output_vexpr_ctxs; // If the result file format is plain text, like CSV, this _file_writer is owned by this FileResultWriter. // If the result file format is Parquet, this _file_writer is owned by _parquet_writer. @@ -119,7 +130,6 @@ class VFileResultWriter final : public ResultWriter { // the suffix idx of export file name, start at 0 int _file_idx = 0; - RuntimeProfile* _parent_profile; // profile from result sink, not owned // total time cost on append batch operation RuntimeProfile::Counter* _append_row_batch_timer = nullptr; // tuple convert timer, child timer of _append_row_batch_timer @@ -142,5 +152,9 @@ class VFileResultWriter final : public ResultWriter { RowDescriptor _output_row_descriptor; // parquet/orc file writer std::unique_ptr _vfile_writer; + + std::string_view _header_type; + std::string_view _header; + std::unique_ptr _writer; }; } // namespace doris::vectorized diff --git a/be/src/vec/sink/writer/vjdbc_table_writer.h b/be/src/vec/sink/writer/vjdbc_table_writer.h index 1aa1e59482d43d8..205f0835f5dbf60 100644 --- a/be/src/vec/sink/writer/vjdbc_table_writer.h +++ b/be/src/vec/sink/writer/vjdbc_table_writer.h @@ -50,6 +50,8 @@ class VJdbcTableWriter final : public AsyncResultWriter, public JdbcConnector { bool in_transaction() override { return TableConnector::_is_in_transaction; } + Status commit_trans() override { return JdbcConnector::finish_trans(); } + private: JdbcConnectorParam _param; }; diff --git a/be/src/vec/sink/writer/vmysql_table_writer.cpp b/be/src/vec/sink/writer/vmysql_table_writer.cpp index a9dad2ca800d735..6897a6b3e24b657 100644 --- a/be/src/vec/sink/writer/vmysql_table_writer.cpp +++ b/be/src/vec/sink/writer/vmysql_table_writer.cpp @@ -115,12 +115,12 @@ Status VMysqlTableWriter::append_block(vectorized::Block& block) { RETURN_IF_ERROR(_projection_block(block, &output_block)); auto num_rows = output_block.rows(); for (int i = 0; i < num_rows; ++i) { - RETURN_IF_ERROR(insert_row(output_block, i)); + RETURN_IF_ERROR(_insert_row(output_block, i)); } return Status::OK(); } -Status VMysqlTableWriter::insert_row(vectorized::Block& block, size_t row) { +Status VMysqlTableWriter::_insert_row(vectorized::Block& block, size_t row) { _insert_stmt_buffer.clear(); fmt::format_to(_insert_stmt_buffer, "INSERT INTO {} VALUES (", _conn_info.table_name); int num_columns = _vec_output_expr_ctxs.size(); diff --git a/be/src/vec/sink/writer/vmysql_table_writer.h b/be/src/vec/sink/writer/vmysql_table_writer.h index bb134f9a581a8c8..9028801144ddeb5 100644 --- a/be/src/vec/sink/writer/vmysql_table_writer.h +++ b/be/src/vec/sink/writer/vmysql_table_writer.h @@ -53,12 +53,10 @@ class VMysqlTableWriter final : public AsyncResultWriter { Status append_block(vectorized::Block& block) override; - Status finish_trans() { return Status::OK(); } - Status close() override; private: - Status insert_row(vectorized::Block& block, size_t row); + Status _insert_row(vectorized::Block& block, size_t row); MysqlConnInfo _conn_info; fmt::memory_buffer _insert_stmt_buffer; MYSQL* _mysql_conn; diff --git a/be/src/vec/sink/writer/vodbc_table_writer.h b/be/src/vec/sink/writer/vodbc_table_writer.h index e07f44c9e496d6d..3df973e4b1253fe 100644 --- a/be/src/vec/sink/writer/vodbc_table_writer.h +++ b/be/src/vec/sink/writer/vodbc_table_writer.h @@ -1,4 +1,3 @@ - // 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 @@ -50,6 +49,8 @@ class VOdbcTableWriter final : public AsyncResultWriter, public ODBCConnector { Status close() override { return ODBCConnector::close(); } bool in_transaction() override { return TableConnector::_is_in_transaction; } + + Status commit_trans() override { return ODBCConnector::finish_trans(); } }; } // namespace vectorized } // namespace doris \ No newline at end of file diff --git a/be/src/vec/utils/template_helpers.hpp b/be/src/vec/utils/template_helpers.hpp index fa94180cde5a93d..b2b466466217f21 100644 --- a/be/src/vec/utils/template_helpers.hpp +++ b/be/src/vec/utils/template_helpers.hpp @@ -48,4 +48,12 @@ std::variant inline make_bool_variant(bool cond } } +template +struct Overload : Callables... { + using Callables::operator()...; +}; + +template +Overload(Callables&&... callables) -> Overload; + } // namespace doris::vectorized diff --git a/be/test/io/fs/stream_sink_file_writer_test.cpp b/be/test/io/fs/stream_sink_file_writer_test.cpp index e894cbaf912c333..c6ac4a3f501ff27 100644 --- a/be/test/io/fs/stream_sink_file_writer_test.cpp +++ b/be/test/io/fs/stream_sink_file_writer_test.cpp @@ -24,6 +24,7 @@ #include "olap/olap_common.h" #include "util/debug/leakcheck_disabler.h" #include "util/faststring.h" +#include "vec/sink/load_stream_stub.h" namespace doris { @@ -35,134 +36,74 @@ namespace doris { } while (false) #endif -DEFINE_string(connection_type, "", "Connection type. Available values: single, pooled, short"); -DEFINE_int32(timeout_ms, 100, "RPC timeout in milliseconds"); -DEFINE_int32(max_retry, 3, "Max retries(not including the first RPC)"); -DEFINE_int32(idle_timeout_s, -1, - "Connection will be closed if there is no " - "read/write operations during the last `idle_timeout_s'"); +constexpr int64_t LOAD_ID_LO = 1; +constexpr int64_t LOAD_ID_HI = 2; +constexpr int64_t NUM_STREAM = 3; +constexpr int64_t PARTITION_ID = 1234; +constexpr int64_t INDEX_ID = 2345; +constexpr int64_t TABLET_ID = 3456; +constexpr int32_t SEGMENT_ID = 4567; +const std::string DATA0 = "segment data"; +const std::string DATA1 = "hello world"; + +static std::atomic g_num_request; class StreamSinkFileWriterTest : public testing::Test { - class MockStreamSinkFileRecevier : public brpc::StreamInputHandler { + class MockStreamStub : public LoadStreamStub { public: - virtual int on_received_messages(brpc::StreamId id, butil::IOBuf* const messages[], - size_t size) { - std::stringstream str; - for (size_t i = 0; i < size; ++i) { - str << "msg[" << i << "]=" << *messages[i]; + MockStreamStub(PUniqueId load_id, int64_t src_id) : LoadStreamStub(load_id, src_id) {}; + + virtual ~MockStreamStub() = default; + + // APPEND_DATA + virtual Status append_data(int64_t partition_id, int64_t index_id, int64_t tablet_id, + int64_t segment_id, std::span data, + bool segment_eos = false) override { + EXPECT_EQ(PARTITION_ID, partition_id); + EXPECT_EQ(INDEX_ID, index_id); + EXPECT_EQ(TABLET_ID, tablet_id); + EXPECT_EQ(SEGMENT_ID, segment_id); + if (segment_eos) { + EXPECT_EQ(0, data.size()); + } else { + EXPECT_EQ(2, data.size()); + EXPECT_EQ(DATA0, data[0].to_string()); + EXPECT_EQ(DATA1, data[1].to_string()); } - LOG(INFO) << "Received from Stream=" << id << ": " << str.str(); - return 0; - } - virtual void on_idle_timeout(brpc::StreamId id) { - LOG(INFO) << "Stream=" << id << " has no data transmission for a while"; - } - virtual void on_closed(brpc::StreamId id) { LOG(INFO) << "Stream=" << id << " is closed"; } - }; - - class StreamingSinkFileService : public PBackendService { - public: - StreamingSinkFileService() : _sd(brpc::INVALID_STREAM_ID) {} - virtual ~StreamingSinkFileService() { brpc::StreamClose(_sd); }; - virtual void open_stream_sink(google::protobuf::RpcController* controller, - const POpenStreamSinkRequest*, - POpenStreamSinkResponse* response, - google::protobuf::Closure* done) { - brpc::ClosureGuard done_guard(done); - - brpc::Controller* cntl = static_cast(controller); - brpc::StreamOptions stream_options; - stream_options.handler = &_receiver; - CHECK_EQ(0, brpc::StreamAccept(&_sd, *cntl, &stream_options)); - Status::OK().to_protobuf(response->mutable_status()); + g_num_request++; + return Status::OK(); } - - private: - MockStreamSinkFileRecevier _receiver; - brpc::StreamId _sd; }; public: - StreamSinkFileWriterTest() { srand(time(nullptr)); } - ~StreamSinkFileWriterTest() {} + StreamSinkFileWriterTest() = default; + ~StreamSinkFileWriterTest() = default; protected: virtual void SetUp() { - // init channel - brpc::Channel channel; - brpc::ChannelOptions options; - options.protocol = brpc::PROTOCOL_BAIDU_STD; - options.connection_type = FLAGS_connection_type; - options.timeout_ms = FLAGS_timeout_ms; - options.max_retry = FLAGS_max_retry; - std::stringstream port; - CHECK_EQ(0, channel.Init("127.0.0.1:18946", nullptr)); - - // init server - _stream_service = new StreamingSinkFileService(); - CHECK_EQ(0, _server.AddService(_stream_service, brpc::SERVER_DOESNT_OWN_SERVICE)); - brpc::ServerOptions server_options; - server_options.idle_timeout_sec = FLAGS_idle_timeout_s; - { - debug::ScopedLeakCheckDisabler disable_lsan; - CHECK_EQ(0, _server.Start("127.0.0.1:18946", &server_options)); + _load_id.set_hi(LOAD_ID_HI); + _load_id.set_lo(LOAD_ID_LO); + for (int src_id = 0; src_id < NUM_STREAM; src_id++) { + _streams.emplace_back(new MockStreamStub(_load_id, src_id)); } - - // init stream connect - PBackendService_Stub stub(&channel); - brpc::Controller cntl; - brpc::StreamId stream; - CHECK_EQ(0, brpc::StreamCreate(&stream, cntl, NULL)); - - POpenStreamSinkRequest request; - POpenStreamSinkResponse response; - request.mutable_load_id()->set_hi(1); - request.mutable_load_id()->set_lo(1); - stub.open_stream_sink(&cntl, &request, &response, NULL); - - brpc::Join(cntl.call_id()); - _stream = stream; } - virtual void TearDown() { - CHECK_EQ(0, brpc::StreamClose(_stream)); - CHECK_EQ(0, _server.Stop(1000)); - CHECK_EQ(0, _server.Join()); - delete _stream_service; - } + virtual void TearDown() {} - StreamingSinkFileService* _stream_service; - brpc::StreamId _stream; - brpc::Server _server; + PUniqueId _load_id; + std::vector> _streams; }; -TEST_F(StreamSinkFileWriterTest, TestInit) { - std::vector stream_ids {_stream}; - io::StreamSinkFileWriter writer(0, stream_ids); - PUniqueId load_id; - load_id.set_hi(1); - load_id.set_lo(2); - writer.init(load_id, 3, 4, 5, 6); -} +TEST_F(StreamSinkFileWriterTest, Test) { + g_num_request = 0; + io::StreamSinkFileWriter writer(_streams); + writer.init(_load_id, PARTITION_ID, INDEX_ID, TABLET_ID, SEGMENT_ID); + std::vector slices {DATA0, DATA1}; -TEST_F(StreamSinkFileWriterTest, TestAppend) { - std::vector stream_ids {_stream}; - io::StreamSinkFileWriter writer(0, stream_ids); - PUniqueId load_id; - load_id.set_hi(1); - load_id.set_lo(2); - writer.init(load_id, 3, 4, 5, 6); - std::vector slices {"hello"}; - CHECK_STATUS_OK(writer.appendv(&slices[0], slices.size())); -} - -TEST_F(StreamSinkFileWriterTest, TestFinalize) { - std::vector stream_ids {_stream}; - io::StreamSinkFileWriter writer(0, stream_ids); - PUniqueId load_id; - load_id.set_hi(1); - load_id.set_lo(2); - writer.init(load_id, 3, 4, 5, 6); + CHECK_STATUS_OK(writer.appendv(&(*slices.begin()), slices.size())); + EXPECT_EQ(NUM_STREAM, g_num_request); CHECK_STATUS_OK(writer.finalize()); + EXPECT_EQ(NUM_STREAM * 2, g_num_request); } + } // namespace doris diff --git a/be/test/olap/delete_handler_test.cpp b/be/test/olap/delete_handler_test.cpp index c2aa5f1efebca58..605e3e389c133e9 100644 --- a/be/test/olap/delete_handler_test.cpp +++ b/be/test/olap/delete_handler_test.cpp @@ -56,7 +56,7 @@ namespace doris { using namespace ErrorCode; static const uint32_t MAX_PATH_LEN = 1024; -static StorageEngine* k_engine = nullptr; +static std::unique_ptr k_engine; static void set_up() { char buffer[MAX_PATH_LEN]; @@ -90,11 +90,7 @@ static void tear_down() { EXPECT_TRUE(io::global_local_filesystem() ->delete_directory(string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX) .ok()); - if (k_engine != nullptr) { - k_engine->stop(); - delete k_engine; - k_engine = nullptr; - } + k_engine.reset(); } static void set_default_create_tablet_request(TCreateTabletReq* request) { @@ -393,6 +389,33 @@ TEST_F(TestDeleteConditionHandler, StoreCondSucceed) { EXPECT_STREQ("k12!='9'", del_pred.sub_predicates(5).c_str()); EXPECT_STREQ("k$1>>'1'", del_pred.sub_predicates(6).c_str()); + // check sub predicate v2 + + EXPECT_EQ(size_t(7), del_pred.sub_predicates_v2_size()); + EXPECT_STREQ("k1", del_pred.sub_predicates_v2(0).column_name().c_str()); + EXPECT_STREQ("k2", del_pred.sub_predicates_v2(1).column_name().c_str()); + EXPECT_STREQ("k3", del_pred.sub_predicates_v2(2).column_name().c_str()); + EXPECT_STREQ("k4", del_pred.sub_predicates_v2(3).column_name().c_str()); + EXPECT_STREQ("k5", del_pred.sub_predicates_v2(4).column_name().c_str()); + EXPECT_STREQ("k12", del_pred.sub_predicates_v2(5).column_name().c_str()); + EXPECT_STREQ("k$1", del_pred.sub_predicates_v2(6).column_name().c_str()); + + EXPECT_STREQ("=", del_pred.sub_predicates_v2(0).op().c_str()); + EXPECT_STREQ(">>", del_pred.sub_predicates_v2(1).op().c_str()); + EXPECT_STREQ("<=", del_pred.sub_predicates_v2(2).op().c_str()); + EXPECT_STREQ("IS", del_pred.sub_predicates_v2(3).op().c_str()); + EXPECT_STREQ("=", del_pred.sub_predicates_v2(4).op().c_str()); + EXPECT_STREQ("!=", del_pred.sub_predicates_v2(5).op().c_str()); + EXPECT_STREQ(">>", del_pred.sub_predicates_v2(6).op().c_str()); + + EXPECT_STREQ("1", del_pred.sub_predicates_v2(0).cond_value().c_str()); + EXPECT_STREQ("3", del_pred.sub_predicates_v2(1).cond_value().c_str()); + EXPECT_STREQ("5", del_pred.sub_predicates_v2(2).cond_value().c_str()); + EXPECT_STREQ("NULL", del_pred.sub_predicates_v2(3).cond_value().c_str()); + EXPECT_STREQ("7", del_pred.sub_predicates_v2(4).cond_value().c_str()); + EXPECT_STREQ("9", del_pred.sub_predicates_v2(5).cond_value().c_str()); + EXPECT_STREQ("1", del_pred.sub_predicates_v2(6).cond_value().c_str()); + EXPECT_EQ(size_t(1), del_pred.in_predicates_size()); EXPECT_FALSE(del_pred.in_predicates(0).is_not_in()); EXPECT_STREQ("k13", del_pred.in_predicates(0).column_name().c_str()); diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp index 511904debbaf7fe..a8dfa4a7609b39c 100644 --- a/be/test/olap/delta_writer_test.cpp +++ b/be/test/olap/delta_writer_test.cpp @@ -67,7 +67,7 @@ class OlapMeta; static const uint32_t MAX_PATH_LEN = 1024; -static StorageEngine* k_engine = nullptr; +static std::unique_ptr k_engine; static void set_up() { char buffer[MAX_PATH_LEN]; @@ -83,7 +83,6 @@ static void set_up() { EXPECT_TRUE(s.ok()) << s.to_string(); ExecEnv* exec_env = doris::ExecEnv::GetInstance(); - exec_env->set_storage_engine(k_engine); exec_env->set_memtable_memory_limiter(new MemTableMemoryLimiter()); k_engine->start_bg_threads(); } @@ -91,11 +90,7 @@ static void set_up() { static void tear_down() { ExecEnv* exec_env = doris::ExecEnv::GetInstance(); exec_env->set_memtable_memory_limiter(nullptr); - if (k_engine != nullptr) { - k_engine->stop(); - delete k_engine; - k_engine = nullptr; - } + k_engine.reset(); EXPECT_EQ(system("rm -rf ./data_test"), 0); io::global_local_filesystem()->delete_directory(std::string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX); @@ -668,8 +663,8 @@ TEST_F(TestDeltaWriter, vec_write) { RowsetSharedPtr rowset = tablet_rs.second; TabletPublishStatistics stats; res = k_engine->txn_manager()->publish_txn(meta, write_req.partition_id, write_req.txn_id, - write_req.tablet_id, write_req.schema_hash, - tablet_rs.first.tablet_uid, version, &stats); + write_req.tablet_id, tablet_rs.first.tablet_uid, + version, &stats); ASSERT_TRUE(res.ok()); std::cout << "start to add inc rowset:" << rowset->rowset_id() << ", num rows:" << rowset->num_rows() << ", version:" << rowset->version().first @@ -765,7 +760,7 @@ TEST_F(TestDeltaWriter, vec_sequence_col) { TabletPublishStatistics pstats; res = k_engine->txn_manager()->publish_txn( meta, write_req.partition_id, write_req.txn_id, write_req.tablet_id, - write_req.schema_hash, tablet_related_rs.begin()->first.tablet_uid, version, &pstats); + tablet_related_rs.begin()->first.tablet_uid, version, &pstats); ASSERT_TRUE(res.ok()); std::cout << "start to add inc rowset:" << rowset->rowset_id() << ", num rows:" << rowset->num_rows() << ", version:" << rowset->version().first @@ -914,10 +909,9 @@ TEST_F(TestDeltaWriter, vec_sequence_col_concurrent_write) { std::cout << "start to publish txn" << std::endl; rowset1 = tablet_related_rs.begin()->second; TabletPublishStatistics pstats; - res = k_engine->txn_manager()->publish_txn(meta, write_req.partition_id, write_req.txn_id, - write_req.tablet_id, write_req.schema_hash, - tablet_related_rs.begin()->first.tablet_uid, - version, &pstats); + res = k_engine->txn_manager()->publish_txn( + meta, write_req.partition_id, write_req.txn_id, write_req.tablet_id, + tablet_related_rs.begin()->first.tablet_uid, version, &pstats); ASSERT_TRUE(res.ok()); std::cout << "start to add inc rowset:" << rowset1->rowset_id() << ", num rows:" << rowset1->num_rows() @@ -968,10 +962,9 @@ TEST_F(TestDeltaWriter, vec_sequence_col_concurrent_write) { ASSERT_TRUE(delete_bitmap->contains({rowset2->rowset_id(), 0, 0}, 1)); TabletPublishStatistics pstats; - res = k_engine->txn_manager()->publish_txn(meta, write_req.partition_id, write_req.txn_id, - write_req.tablet_id, write_req.schema_hash, - tablet_related_rs.begin()->first.tablet_uid, - version, &pstats); + res = k_engine->txn_manager()->publish_txn( + meta, write_req.partition_id, write_req.txn_id, write_req.tablet_id, + tablet_related_rs.begin()->first.tablet_uid, version, &pstats); ASSERT_TRUE(res.ok()); std::cout << "start to add inc rowset:" << rowset2->rowset_id() << ", num rows:" << rowset2->num_rows() diff --git a/be/test/olap/engine_storage_migration_task_test.cpp b/be/test/olap/engine_storage_migration_task_test.cpp index 2dca95432190c69..706cb7497966dd7 100644 --- a/be/test/olap/engine_storage_migration_task_test.cpp +++ b/be/test/olap/engine_storage_migration_task_test.cpp @@ -58,7 +58,7 @@ class OlapMeta; static const uint32_t MAX_PATH_LEN = 1024; -static StorageEngine* k_engine = nullptr; +static std::unique_ptr k_engine; static std::string path1; static std::string path2; @@ -80,7 +80,6 @@ static void set_up() { Status s = doris::StorageEngine::open(options, &k_engine); EXPECT_TRUE(s.ok()) << s.to_string(); ExecEnv* exec_env = doris::ExecEnv::GetInstance(); - exec_env->set_storage_engine(k_engine); k_engine->start_bg_threads(); exec_env->set_memtable_memory_limiter(new MemTableMemoryLimiter()); } @@ -88,11 +87,7 @@ static void set_up() { static void tear_down() { ExecEnv* exec_env = doris::ExecEnv::GetInstance(); exec_env->set_memtable_memory_limiter(nullptr); - if (k_engine != nullptr) { - k_engine->stop(); - delete k_engine; - k_engine = nullptr; - } + k_engine.reset(); EXPECT_EQ(system("rm -rf ./data_test_1"), 0); EXPECT_EQ(system("rm -rf ./data_test_2"), 0); EXPECT_TRUE(io::global_local_filesystem() @@ -224,8 +219,8 @@ TEST_F(TestEngineStorageMigrationTask, write_and_migration) { RowsetSharedPtr rowset = tablet_rs.second; TabletPublishStatistics stats; res = k_engine->txn_manager()->publish_txn(meta, write_req.partition_id, write_req.txn_id, - tablet->tablet_id(), tablet->schema_hash(), - tablet->tablet_uid(), version, &stats); + tablet->tablet_id(), tablet->tablet_uid(), + version, &stats); EXPECT_EQ(Status::OK(), res); res = tablet->add_inc_rowset(rowset); EXPECT_EQ(Status::OK(), res); diff --git a/be/test/olap/memtable_flush_executor_test.cpp b/be/test/olap/memtable_flush_executor_test.cpp index 50d6003088d1240..efe95e36a3555a3 100644 --- a/be/test/olap/memtable_flush_executor_test.cpp +++ b/be/test/olap/memtable_flush_executor_test.cpp @@ -40,8 +40,8 @@ namespace doris { -inline StorageEngine* k_engine = nullptr; -MemTableFlushExecutor* k_flush_executor = nullptr; +static std::unique_ptr k_engine; +static MemTableFlushExecutor* k_flush_executor = nullptr; void set_up() { char buffer[1024]; @@ -58,15 +58,11 @@ void set_up() { Status s = doris::StorageEngine::open(options, &k_engine); EXPECT_TRUE(s.ok()) << s.to_string(); - ExecEnv* exec_env = doris::ExecEnv::GetInstance(); - exec_env->set_storage_engine(k_engine); - k_flush_executor = k_engine->memtable_flush_executor(); } void tear_down() { - delete k_engine; - k_engine = nullptr; + k_engine.reset(); system("rm -rf ./flush_test"); EXPECT_TRUE(io::global_local_filesystem() ->delete_directory(std::string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX) diff --git a/be/test/olap/memtable_memory_limiter_test.cpp b/be/test/olap/memtable_memory_limiter_test.cpp index dca8544af3e54aa..187fc09cb4d07fa 100644 --- a/be/test/olap/memtable_memory_limiter_test.cpp +++ b/be/test/olap/memtable_memory_limiter_test.cpp @@ -87,7 +87,6 @@ class MemTableMemoryLimiterTest : public testing::Test { options.store_paths = paths; Status s = doris::StorageEngine::open(options, &_engine); ExecEnv* exec_env = doris::ExecEnv::GetInstance(); - exec_env->set_storage_engine(_engine); _engine->start_bg_threads(); exec_env->set_memtable_memory_limiter(new MemTableMemoryLimiter()); } @@ -95,17 +94,12 @@ class MemTableMemoryLimiterTest : public testing::Test { void TearDown() override { ExecEnv* exec_env = doris::ExecEnv::GetInstance(); exec_env->set_memtable_memory_limiter(nullptr); - if (_engine != nullptr) { - _engine->stop(); - delete _engine; - _engine = nullptr; - } EXPECT_EQ(system("rm -rf ./data_test"), 0); io::global_local_filesystem()->delete_directory(std::string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX); } - StorageEngine* _engine = nullptr; + std::unique_ptr _engine; }; TEST_F(MemTableMemoryLimiterTest, handle_memtable_flush_test) { diff --git a/be/test/olap/remote_rowset_gc_test.cpp b/be/test/olap/remote_rowset_gc_test.cpp index 2028398dc2fe0fd..28d69fe8a74e9d2 100644 --- a/be/test/olap/remote_rowset_gc_test.cpp +++ b/be/test/olap/remote_rowset_gc_test.cpp @@ -61,7 +61,7 @@ namespace doris { class OlapMeta; -static StorageEngine* k_engine = nullptr; +static std::unique_ptr k_engine; static const std::string kTestDir = "./ut_dir/remote_rowset_gc_test"; static constexpr int64_t kResourceId = 10000; @@ -105,13 +105,7 @@ class RemoteRowsetGcTest : public testing::Test { doris::StorageEngine::open(options, &k_engine); } - static void TearDownTestSuite() { - if (k_engine != nullptr) { - k_engine->stop(); - delete k_engine; - k_engine = nullptr; - } - } + static void TearDownTestSuite() { k_engine.reset(); } }; static void create_tablet_request_with_sequence_col(int64_t tablet_id, int32_t schema_hash, diff --git a/be/test/olap/rowset/beta_rowset_test.cpp b/be/test/olap/rowset/beta_rowset_test.cpp index ab984423e8860d5..5e259745d68f434 100644 --- a/be/test/olap/rowset/beta_rowset_test.cpp +++ b/be/test/olap/rowset/beta_rowset_test.cpp @@ -73,7 +73,7 @@ namespace doris { using namespace ErrorCode; static const uint32_t MAX_PATH_LEN = 1024; -inline StorageEngine* k_engine = nullptr; +static std::unique_ptr k_engine; static const std::string kTestDir = "./data_test/data/beta_rowset_test"; class BetaRowsetTest : public testing::Test { @@ -103,19 +103,10 @@ class BetaRowsetTest : public testing::Test { Status s = doris::StorageEngine::open(options, &k_engine); EXPECT_TRUE(s.ok()) << s.to_string(); - ExecEnv* exec_env = doris::ExecEnv::GetInstance(); - exec_env->set_storage_engine(k_engine); - EXPECT_TRUE(io::global_local_filesystem()->create_directory(kTestDir).ok()); } - static void TearDownTestSuite() { - if (k_engine != nullptr) { - k_engine->stop(); - delete k_engine; - k_engine = nullptr; - } - } + static void TearDownTestSuite() { k_engine.reset(); } protected: OlapReaderStatistics _stats; diff --git a/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp b/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp index 2d79776c3b86bf2..2e803e4ff1f5bcb 100644 --- a/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp +++ b/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp @@ -80,8 +80,8 @@ void get_bitmap_reader_iter(const std::string& file_name, const ColumnIndexMetaP BitmapIndexReader** reader, BitmapIndexIterator** iter) { io::FileReaderSPtr file_reader; ASSERT_EQ(io::global_local_filesystem()->open_file(file_name, &file_reader), Status::OK()); - *reader = new BitmapIndexReader(std::move(file_reader), &meta.bitmap_index()); - auto st = (*reader)->load(true, false); + *reader = new BitmapIndexReader(std::move(file_reader)); + auto st = (*reader)->load(true, false, &meta.bitmap_index()); EXPECT_TRUE(st.ok()); st = (*reader)->new_iterator(iter); diff --git a/be/test/olap/rowset/segment_v2/ordinal_page_index_test.cpp b/be/test/olap/rowset/segment_v2/ordinal_page_index_test.cpp index d10c1dde836ed22..a5024f4147492ec 100644 --- a/be/test/olap/rowset/segment_v2/ordinal_page_index_test.cpp +++ b/be/test/olap/rowset/segment_v2/ordinal_page_index_test.cpp @@ -70,8 +70,8 @@ TEST_F(OrdinalPageIndexTest, normal) { io::FileReaderSPtr file_reader; EXPECT_TRUE(fs->open_file(filename, &file_reader).ok()); - OrdinalIndexReader index(file_reader, &index_meta.ordinal_index(), 16 * 1024 * 4096 + 1); - EXPECT_TRUE(index.load(true, false).ok()); + OrdinalIndexReader index(file_reader, 16 * 1024 * 4096 + 1); + EXPECT_TRUE(index.load(true, false, &index_meta.ordinal_index()).ok()); EXPECT_EQ(16 * 1024, index.num_data_pages()); EXPECT_EQ(1, index.get_first_ordinal(0)); EXPECT_EQ(4096, index.get_last_ordinal(0)); @@ -124,8 +124,8 @@ TEST_F(OrdinalPageIndexTest, one_data_page) { EXPECT_EQ(data_page_pointer, root_page_pointer); } - OrdinalIndexReader index(nullptr, &index_meta.ordinal_index(), num_values); - EXPECT_TRUE(index.load(true, false).ok()); + OrdinalIndexReader index(nullptr, num_values); + EXPECT_TRUE(index.load(true, false, &index_meta.ordinal_index()).ok()); EXPECT_EQ(1, index.num_data_pages()); EXPECT_EQ(0, index.get_first_ordinal(0)); EXPECT_EQ(num_values - 1, index.get_last_ordinal(0)); diff --git a/be/test/olap/rowset/segment_v2/zone_map_index_test.cpp b/be/test/olap/rowset/segment_v2/zone_map_index_test.cpp index 1d9345f908522c2..22b69c5cae76df3 100644 --- a/be/test/olap/rowset/segment_v2/zone_map_index_test.cpp +++ b/be/test/olap/rowset/segment_v2/zone_map_index_test.cpp @@ -81,8 +81,8 @@ class ColumnZoneMapTest : public testing::Test { io::FileReaderSPtr file_reader; EXPECT_TRUE(fs->open_file(filename, &file_reader).ok()); - ZoneMapIndexReader column_zone_map(file_reader, &index_meta.zone_map_index()); - Status status = column_zone_map.load(true, false); + ZoneMapIndexReader column_zone_map(file_reader); + Status status = column_zone_map.load(true, false, &index_meta.zone_map_index()); EXPECT_TRUE(status.ok()); EXPECT_EQ(3, column_zone_map.num_pages()); const std::vector& zone_maps = column_zone_map.page_zone_maps(); @@ -128,8 +128,8 @@ class ColumnZoneMapTest : public testing::Test { io::FileReaderSPtr file_reader; EXPECT_TRUE(fs->open_file(filename, &file_reader).ok()); - ZoneMapIndexReader column_zone_map(file_reader, &index_meta.zone_map_index()); - Status status = column_zone_map.load(true, false); + ZoneMapIndexReader column_zone_map(file_reader); + Status status = column_zone_map.load(true, false, &index_meta.zone_map_index()); EXPECT_TRUE(status.ok()); EXPECT_EQ(1, column_zone_map.num_pages()); const std::vector& zone_maps = column_zone_map.page_zone_maps(); @@ -181,8 +181,8 @@ TEST_F(ColumnZoneMapTest, NormalTestIntPage) { io::FileReaderSPtr file_reader; EXPECT_TRUE(fs->open_file(filename, &file_reader).ok()); - ZoneMapIndexReader column_zone_map(file_reader, &index_meta.zone_map_index()); - Status status = column_zone_map.load(true, false); + ZoneMapIndexReader column_zone_map(file_reader); + Status status = column_zone_map.load(true, false, &index_meta.zone_map_index()); EXPECT_TRUE(status.ok()); EXPECT_EQ(3, column_zone_map.num_pages()); const std::vector& zone_maps = column_zone_map.page_zone_maps(); diff --git a/be/test/olap/segcompaction_test.cpp b/be/test/olap/segcompaction_test.cpp index 468c6482cb040a6..ed0b51341bd465d 100644 --- a/be/test/olap/segcompaction_test.cpp +++ b/be/test/olap/segcompaction_test.cpp @@ -47,7 +47,7 @@ namespace doris { using namespace ErrorCode; static const uint32_t MAX_PATH_LEN = 1024; -StorageEngine* l_engine = nullptr; +static std::unique_ptr l_engine; static const std::string lTestDir = "./data_test/data/segcompaction_test"; class SegCompactionTest : public testing::Test { @@ -79,7 +79,6 @@ class SegCompactionTest : public testing::Test { EXPECT_TRUE(s.ok()) << s.to_string(); ExecEnv* exec_env = doris::ExecEnv::GetInstance(); - exec_env->set_storage_engine(l_engine); EXPECT_TRUE(io::global_local_filesystem()->create_directory(lTestDir).ok()); @@ -87,11 +86,7 @@ class SegCompactionTest : public testing::Test { } void TearDown() { - if (l_engine != nullptr) { - l_engine->stop(); - delete l_engine; - l_engine = nullptr; - } + l_engine.reset(); config::enable_segcompaction = false; } diff --git a/be/test/olap/tablet_cooldown_test.cpp b/be/test/olap/tablet_cooldown_test.cpp index 15e78ecc4e8e0ee..ff248267630d7c4 100644 --- a/be/test/olap/tablet_cooldown_test.cpp +++ b/be/test/olap/tablet_cooldown_test.cpp @@ -71,7 +71,7 @@ namespace doris { class OlapMeta; struct Slice; -static StorageEngine* k_engine = nullptr; +static std::unique_ptr k_engine; static const std::string kTestDir = "ut_dir/tablet_cooldown_test"; static constexpr int64_t kResourceId = 10000; @@ -268,11 +268,7 @@ class TabletCooldownTest : public testing::Test { static void TearDownTestSuite() { ExecEnv* exec_env = doris::ExecEnv::GetInstance(); exec_env->set_memtable_memory_limiter(nullptr); - if (k_engine != nullptr) { - k_engine->stop(); - delete k_engine; - k_engine = nullptr; - } + k_engine.reset(); } }; @@ -339,15 +335,15 @@ static TDescriptorTable create_descriptor_tablet_with_sequence_col() { return desc_tbl_builder.desc_tbl(); } -void createTablet(StorageEngine* engine, TabletSharedPtr* tablet, int64_t replica_id, - int32_t schema_hash, int64_t tablet_id, int64_t txn_id, int64_t partition_id) { +void createTablet(TabletSharedPtr* tablet, int64_t replica_id, int32_t schema_hash, + int64_t tablet_id, int64_t txn_id, int64_t partition_id) { // create tablet std::unique_ptr profile; profile = std::make_unique("CreateTablet"); TCreateTabletReq request; create_tablet_request_with_sequence_col(tablet_id, schema_hash, &request); request.__set_replica_id(replica_id); - Status st = engine->create_tablet(request, profile.get()); + Status st = k_engine->create_tablet(request, profile.get()); ASSERT_EQ(Status::OK(), st); TDescriptorTable tdesc_tbl = create_descriptor_tablet_with_sequence_col(); @@ -414,20 +410,20 @@ void createTablet(StorageEngine* engine, TabletSharedPtr* tablet, int64_t replic delete delta_writer; // publish version success - *tablet = engine->tablet_manager()->get_tablet(write_req.tablet_id, write_req.schema_hash); + *tablet = k_engine->tablet_manager()->get_tablet(write_req.tablet_id, write_req.schema_hash); OlapMeta* meta = (*tablet)->data_dir()->get_meta(); Version version; version.first = (*tablet)->rowset_with_max_version()->end_version() + 1; version.second = (*tablet)->rowset_with_max_version()->end_version() + 1; std::map tablet_related_rs; - engine->txn_manager()->get_txn_related_tablets(write_req.txn_id, write_req.partition_id, - &tablet_related_rs); + k_engine->txn_manager()->get_txn_related_tablets(write_req.txn_id, write_req.partition_id, + &tablet_related_rs); for (auto& tablet_rs : tablet_related_rs) { RowsetSharedPtr rowset = tablet_rs.second; TabletPublishStatistics stats; - st = engine->txn_manager()->publish_txn(meta, write_req.partition_id, write_req.txn_id, - (*tablet)->tablet_id(), (*tablet)->schema_hash(), - (*tablet)->tablet_uid(), version, &stats); + st = k_engine->txn_manager()->publish_txn(meta, write_req.partition_id, write_req.txn_id, + (*tablet)->tablet_id(), (*tablet)->tablet_uid(), + version, &stats); ASSERT_EQ(Status::OK(), st); st = (*tablet)->add_inc_rowset(rowset); ASSERT_EQ(Status::OK(), st); @@ -438,8 +434,8 @@ void createTablet(StorageEngine* engine, TabletSharedPtr* tablet, int64_t replic TEST_F(TabletCooldownTest, normal) { TabletSharedPtr tablet1; TabletSharedPtr tablet2; - createTablet(k_engine, &tablet1, kReplicaId, kSchemaHash, kTabletId, kTxnId, kPartitionId); - createTablet(k_engine, &tablet2, kReplicaId2, kSchemaHash2, kTabletId2, kTxnId2, kPartitionId2); + createTablet(&tablet1, kReplicaId, kSchemaHash, kTabletId, kTxnId, kPartitionId); + createTablet(&tablet2, kReplicaId2, kSchemaHash2, kTabletId2, kTxnId2, kPartitionId2); // test cooldown tablet1->set_storage_policy_id(kStoragePolicyId); Status st = tablet1->cooldown(); // rowset [0-1] diff --git a/be/test/olap/txn_manager_test.cpp b/be/test/olap/txn_manager_test.cpp index e19bdf9990010a3..2d27576b5a171d6 100644 --- a/be/test/olap/txn_manager_test.cpp +++ b/be/test/olap/txn_manager_test.cpp @@ -175,7 +175,6 @@ class TxnManagerTest : public testing::Test { TPartitionId partition_id = 1123; TTransactionId transaction_id = 111; TTabletId tablet_id = 222; - SchemaHash schema_hash = 333; TabletUid _tablet_uid {0, 0}; PUniqueId load_id; TabletSchemaSPtr _schema; @@ -185,8 +184,8 @@ class TxnManagerTest : public testing::Test { }; TEST_F(TxnManagerTest, PrepareNewTxn) { - Status status = _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid, load_id); + Status status = + _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, _tablet_uid, load_id); EXPECT_TRUE(status == Status::OK()); } @@ -194,10 +193,10 @@ TEST_F(TxnManagerTest, PrepareNewTxn) { // 2. commit txn // 3. should be success TEST_F(TxnManagerTest, CommitTxnWithPrepare) { - Status status = _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid, load_id); - _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash, _tablet_uid, - load_id, _rowset, false); + Status status = + _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, _tablet_uid, load_id); + _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, _tablet_uid, load_id, + _rowset, false); EXPECT_TRUE(status == Status::OK()); RowsetMetaSharedPtr rowset_meta(new RowsetMeta()); status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(), @@ -210,7 +209,7 @@ TEST_F(TxnManagerTest, CommitTxnWithPrepare) { // 2. should success TEST_F(TxnManagerTest, CommitTxnWithNoPrepare) { Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, - schema_hash, _tablet_uid, load_id, _rowset, false); + _tablet_uid, load_id, _rowset, false); EXPECT_TRUE(status == Status::OK()); } @@ -218,10 +217,10 @@ TEST_F(TxnManagerTest, CommitTxnWithNoPrepare) { // 2. should failed TEST_F(TxnManagerTest, CommitTxnTwiceWithDiffRowsetId) { Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, - schema_hash, _tablet_uid, load_id, _rowset, false); + _tablet_uid, load_id, _rowset, false); EXPECT_TRUE(status == Status::OK()); - status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid, load_id, _rowset_diff_id, false); + status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, _tablet_uid, + load_id, _rowset_diff_id, false); EXPECT_TRUE(status != Status::OK()); } @@ -229,30 +228,28 @@ TEST_F(TxnManagerTest, CommitTxnTwiceWithDiffRowsetId) { // 2. should success TEST_F(TxnManagerTest, CommitTxnTwiceWithSameRowsetId) { Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, - schema_hash, _tablet_uid, load_id, _rowset, false); + _tablet_uid, load_id, _rowset, false); EXPECT_TRUE(status == Status::OK()); - status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid, load_id, _rowset_same_id, false); + status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, _tablet_uid, + load_id, _rowset_same_id, false); EXPECT_TRUE(status == Status::OK()); } // 1. prepare twice should be success TEST_F(TxnManagerTest, PrepareNewTxnTwice) { - Status status = _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid, load_id); + Status status = + _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, _tablet_uid, load_id); EXPECT_TRUE(status == Status::OK()); - status = _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid, load_id); + status = _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, _tablet_uid, load_id); EXPECT_TRUE(status == Status::OK()); } // 1. txn could be rollbacked if it is not committed TEST_F(TxnManagerTest, RollbackNotCommittedTxn) { - Status status = _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid, load_id); + Status status = + _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, _tablet_uid, load_id); EXPECT_TRUE(status == Status::OK()); - status = _txn_mgr->rollback_txn(partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid); + status = _txn_mgr->rollback_txn(partition_id, transaction_id, tablet_id, _tablet_uid); EXPECT_TRUE(status == Status::OK()); RowsetMetaSharedPtr rowset_meta(new RowsetMeta()); status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(), @@ -263,10 +260,9 @@ TEST_F(TxnManagerTest, RollbackNotCommittedTxn) { // 1. txn could not be rollbacked if it is committed TEST_F(TxnManagerTest, RollbackCommittedTxn) { Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, - schema_hash, _tablet_uid, load_id, _rowset, false); + _tablet_uid, load_id, _rowset, false); EXPECT_TRUE(status == Status::OK()); - status = _txn_mgr->rollback_txn(partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid); + status = _txn_mgr->rollback_txn(partition_id, transaction_id, tablet_id, _tablet_uid); EXPECT_FALSE(status == Status::OK()); RowsetMetaSharedPtr rowset_meta(new RowsetMeta()); status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(), @@ -278,12 +274,12 @@ TEST_F(TxnManagerTest, RollbackCommittedTxn) { // 1. publish version success TEST_F(TxnManagerTest, PublishVersionSuccessful) { Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, - schema_hash, _tablet_uid, load_id, _rowset, false); + _tablet_uid, load_id, _rowset, false); EXPECT_TRUE(status == Status::OK()); Version new_version(10, 11); TabletPublishStatistics stats; - status = _txn_mgr->publish_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid, new_version, &stats); + status = _txn_mgr->publish_txn(_meta, partition_id, transaction_id, tablet_id, _tablet_uid, + new_version, &stats); EXPECT_TRUE(status == Status::OK()); RowsetMetaSharedPtr rowset_meta(new RowsetMeta()); @@ -302,29 +298,27 @@ TEST_F(TxnManagerTest, PublishNotExistedTxn) { auto not_exist_txn = transaction_id + 1000; TabletPublishStatistics stats; Status status = _txn_mgr->publish_txn(_meta, partition_id, not_exist_txn, tablet_id, - schema_hash, _tablet_uid, new_version, &stats); + _tablet_uid, new_version, &stats); EXPECT_EQ(status, Status::OK()); } TEST_F(TxnManagerTest, DeletePreparedTxn) { - Status status = _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid, load_id); + Status status = + _txn_mgr->prepare_txn(partition_id, transaction_id, tablet_id, _tablet_uid, load_id); EXPECT_TRUE(status == Status::OK()); - status = _txn_mgr->delete_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid); + status = _txn_mgr->delete_txn(_meta, partition_id, transaction_id, tablet_id, _tablet_uid); EXPECT_TRUE(status == Status::OK()); } TEST_F(TxnManagerTest, DeleteCommittedTxn) { Status status = _txn_mgr->commit_txn(_meta, partition_id, transaction_id, tablet_id, - schema_hash, _tablet_uid, load_id, _rowset, false); + _tablet_uid, load_id, _rowset, false); EXPECT_TRUE(status == Status::OK()); RowsetMetaSharedPtr rowset_meta(new RowsetMeta()); status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(), rowset_meta); EXPECT_TRUE(status == Status::OK()); - status = _txn_mgr->delete_txn(_meta, partition_id, transaction_id, tablet_id, schema_hash, - _tablet_uid); + status = _txn_mgr->delete_txn(_meta, partition_id, transaction_id, tablet_id, _tablet_uid); EXPECT_TRUE(status == Status::OK()); RowsetMetaSharedPtr rowset_meta2(new RowsetMeta()); status = RowsetMetaManager::get_rowset_meta(_meta, _tablet_uid, _rowset->rowset_id(), diff --git a/be/test/runtime/load_stream_test.cpp b/be/test/runtime/load_stream_test.cpp index e3d3868547017de..09a21c3a76166db 100644 --- a/be/test/runtime/load_stream_test.cpp +++ b/be/test/runtime/load_stream_test.cpp @@ -51,7 +51,7 @@ using namespace brpc; namespace doris { static const uint32_t MAX_PATH_LEN = 1024; -StorageEngine* z_engine = nullptr; +static std::unique_ptr k_engine; static const std::string zTestDir = "./data_test/data/load_stream_mgr_test"; const int64_t NORMAL_TABLET_ID = 10000; @@ -580,15 +580,14 @@ class LoadStreamMgrTest : public testing::Test { doris::EngineOptions options; options.store_paths = paths; - Status s = doris::StorageEngine::open(options, &z_engine); + Status s = doris::StorageEngine::open(options, &k_engine); EXPECT_TRUE(s.ok()) << s.to_string(); _env = doris::ExecEnv::GetInstance(); - _env->set_storage_engine(z_engine); EXPECT_TRUE(io::global_local_filesystem()->create_directory(zTestDir).ok()); - z_engine->start_bg_threads(); + k_engine->start_bg_threads(); _load_stream_mgr = std::make_unique(4, &_heavy_work_pool, &_light_work_pool); _stream_service = new StreamService(_load_stream_mgr.get()); @@ -604,17 +603,13 @@ class LoadStreamMgrTest : public testing::Test { TCreateTabletReq request; create_tablet_request(NORMAL_TABLET_ID + i, SCHEMA_HASH, &request); auto profile = std::make_unique("test"); - Status res = z_engine->create_tablet(request, profile.get()); + Status res = k_engine->create_tablet(request, profile.get()); EXPECT_EQ(Status::OK(), res); } } void TearDown() override { - if (z_engine != nullptr) { - z_engine->stop(); - delete z_engine; - z_engine = nullptr; - } + k_engine.reset(); _server->Stop(1000); _load_stream_mgr.reset(); CHECK_EQ(0, _server->Join()); @@ -622,9 +617,9 @@ class LoadStreamMgrTest : public testing::Test { } std::string read_data(int64_t txn_id, int64_t partition_id, int64_t tablet_id, uint32_t segid) { - auto tablet = z_engine->tablet_manager()->get_tablet(tablet_id); + auto tablet = k_engine->tablet_manager()->get_tablet(tablet_id); std::map tablet_related_rs; - z_engine->txn_manager()->get_txn_related_tablets(txn_id, partition_id, &tablet_related_rs); + k_engine->txn_manager()->get_txn_related_tablets(txn_id, partition_id, &tablet_related_rs); LOG(INFO) << "get txn related tablet, txn_id=" << txn_id << ", tablet_id=" << tablet_id << "partition_id=" << partition_id; for (auto& [tablet, rowset] : tablet_related_rs) { @@ -679,9 +674,8 @@ TEST_F(LoadStreamMgrTest, one_client_abnormal_load) { EXPECT_EQ(g_response_stat.num, 2); EXPECT_EQ(g_response_stat.success_tablet_ids.size(), 1); EXPECT_EQ(g_response_stat.success_tablet_ids[0], NORMAL_TABLET_ID); - EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 1); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } @@ -708,12 +702,11 @@ TEST_F(LoadStreamMgrTest, one_client_abnormal_index) { close_load(client, 0); wait_for_ack(3); - EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 1); EXPECT_EQ(g_response_stat.num, 3); EXPECT_EQ(g_response_stat.success_tablet_ids.size(), 0); EXPECT_EQ(g_response_stat.failed_tablet_ids.size(), 1); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } @@ -743,7 +736,7 @@ TEST_F(LoadStreamMgrTest, one_client_abnormal_sender) { EXPECT_EQ(g_response_stat.success_tablet_ids.size(), 0); EXPECT_EQ(g_response_stat.failed_tablet_ids.size(), 1); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } @@ -773,12 +766,12 @@ TEST_F(LoadStreamMgrTest, one_client_abnormal_tablet) { EXPECT_EQ(g_response_stat.success_tablet_ids.size(), 0); EXPECT_EQ(g_response_stat.failed_tablet_ids.size(), 1); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } -TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_signle_segment0_zero_bytes) { +TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_single_segment0_zero_bytes) { MockSinkClient client; auto st = client.connect_stream(); EXPECT_TRUE(st.ok()); @@ -814,12 +807,12 @@ TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_signle_segment0_zero_b EXPECT_EQ(g_response_stat.failed_tablet_ids.size(), 1); EXPECT_EQ(g_response_stat.failed_tablet_ids[0], NORMAL_TABLET_ID); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } -TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_signle_segment0) { +TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_single_segment0) { MockSinkClient client; auto st = client.connect_stream(); EXPECT_TRUE(st.ok()); @@ -860,12 +853,12 @@ TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_signle_segment0) { auto written_data = read_data(NORMAL_TXN_ID, NORMAL_PARTITION_ID, NORMAL_TABLET_ID, 0); EXPECT_EQ(written_data, data + data); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } -TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_signle_segment_without_eos) { +TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_single_segment_without_eos) { MockSinkClient client; auto st = client.connect_stream(); EXPECT_TRUE(st.ok()); @@ -901,12 +894,12 @@ TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_signle_segment_without EXPECT_EQ(g_response_stat.failed_tablet_ids.size(), 1); EXPECT_EQ(g_response_stat.failed_tablet_ids[0], NORMAL_TABLET_ID); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } -TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_signle_segment1) { +TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_single_segment1) { MockSinkClient client; auto st = client.connect_stream(); EXPECT_TRUE(st.ok()); @@ -944,7 +937,7 @@ TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_signle_segment1) { EXPECT_EQ(g_response_stat.failed_tablet_ids.size(), 1); EXPECT_EQ(g_response_stat.failed_tablet_ids[0], NORMAL_TABLET_ID); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } @@ -997,7 +990,7 @@ TEST_F(LoadStreamMgrTest, one_client_one_index_one_tablet_two_segment) { written_data = read_data(NORMAL_TXN_ID, NORMAL_PARTITION_ID, NORMAL_TABLET_ID, 1); EXPECT_EQ(written_data, data2); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } @@ -1058,7 +1051,7 @@ TEST_F(LoadStreamMgrTest, one_client_one_index_three_tablet) { written_data = read_data(NORMAL_TXN_ID, NORMAL_PARTITION_ID, NORMAL_TABLET_ID + 2, 0); EXPECT_EQ(written_data, data2); - client.disconnect(); + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } @@ -1100,13 +1093,14 @@ TEST_F(LoadStreamMgrTest, two_client_one_index_one_tablet_three_segment) { // duplicated close close_load(clients[1], 1); wait_for_ack(2); - EXPECT_EQ(g_response_stat.num, 2); + // stream closed, no response will be sent + EXPECT_EQ(g_response_stat.num, 1); EXPECT_EQ(g_response_stat.success_tablet_ids.size(), 0); EXPECT_EQ(g_response_stat.failed_tablet_ids.size(), 0); close_load(clients[0], 0); - wait_for_ack(3); - EXPECT_EQ(g_response_stat.num, 3); + wait_for_ack(2); + EXPECT_EQ(g_response_stat.num, 2); EXPECT_EQ(g_response_stat.success_tablet_ids.size(), 1); EXPECT_EQ(g_response_stat.failed_tablet_ids.size(), 0); EXPECT_EQ(g_response_stat.success_tablet_ids[0], NORMAL_TABLET_ID); @@ -1130,9 +1124,7 @@ TEST_F(LoadStreamMgrTest, two_client_one_index_one_tablet_three_segment) { EXPECT_EQ(written_data, segment_data[sender_id * 3 + i]); } - for (int i = 0; i < 2; i++) { - clients[i].disconnect(); - } + // server will close stream on CLOSE_LOAD wait_for_close(); EXPECT_EQ(_load_stream_mgr->get_load_stream_num(), 0); } diff --git a/be/test/runtime/test_env.cc b/be/test/runtime/test_env.cc deleted file mode 100644 index 39ad186a194bd61..000000000000000 --- a/be/test/runtime/test_env.cc +++ /dev/null @@ -1,97 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "runtime/test_env.h" - -#include -#include -#include -#include -#include - -#include -#include - -#include "common/config.h" -#include "common/status.h" -#include "gtest/gtest_pred_impl.h" -#include "olap/olap_define.h" -#include "olap/options.h" -#include "olap/storage_engine.h" -#include "runtime/exec_env.h" -#include "runtime/result_queue_mgr.h" -#include "runtime/runtime_state.h" -#include "util/uid_util.h" - -namespace doris { - -TestEnv::TestEnv() { - // Some code will use ExecEnv::GetInstance(), so init the global ExecEnv singleton - _exec_env = ExecEnv::GetInstance(); - _exec_env->_result_queue_mgr = new ResultQueueMgr(); - // TODO may need rpc support, etc. -} - -TestEnv::~TestEnv() { - SAFE_DELETE(_exec_env->_result_queue_mgr); - - if (_engine == StorageEngine::_s_instance) { - // the engine instance is created by this test env - StorageEngine::_s_instance = nullptr; - } - SAFE_DELETE(_engine); -} - -void TestEnv::tear_down_query_states() { - _query_states.clear(); -} - -int64_t TestEnv::calculate_mem_tracker(int max_buffers, int block_size) { - DCHECK_GE(max_buffers, -1); - if (max_buffers == -1) { - return -1; - } - return max_buffers * static_cast(block_size); -} - -void TestEnv::init_storage_engine(bool need_open, const std::vector& paths) { - if (StorageEngine::_s_instance) { - LOG(INFO) << "Engine instance already exists"; - return; - } - // init and open storage engine - doris::EngineOptions options; - for (const auto& path : paths) { - options.store_paths.emplace_back(path, -1); - } - options.backend_uid = UniqueId::gen_uid(); - config::tablet_map_shard_size = 1; - config::txn_map_shard_size = 1; - config::txn_shard_size = 1; - - // This engine will be the singleton instance, cuz StorageEngine::_s_instance is nullptr now. - Status st; - if (need_open) { - st = StorageEngine::open(options, &_engine); - } else { - _engine = new StorageEngine(options); - } - EXPECT_TRUE(st.ok()); - _exec_env->set_storage_engine(_engine); -} - -} // end namespace doris diff --git a/be/test/runtime/test_env.h b/be/test/runtime/test_env.h deleted file mode 100644 index c4a3bbedbb2e42e..000000000000000 --- a/be/test/runtime/test_env.h +++ /dev/null @@ -1,63 +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. - -#ifndef DORIS_BE_TEST_QUERY_RUNTIME_TEST_ENV_H -#define DORIS_BE_TEST_QUERY_RUNTIME_TEST_ENV_H - -#include - -#include -#include -#include - -namespace doris { -class ExecEnv; -class RuntimeState; -class StorageEngine; - -/// Helper testing class that creates an environment with runtime memory management -/// similar to the one used by the Doris runtime. Only one TestEnv can be active at a -/// time, because it modifies the global ExecEnv singleton. -class TestEnv { -public: - TestEnv(); - ~TestEnv(); - - // If don't need to open, paths can be empty. - void init_storage_engine(bool need_open, const std::vector& paths = {}); - - // Destroy all RuntimeStates and block managers created by this TestEnv. - void tear_down_query_states(); - - // Calculate memory limit accounting for overflow and negative values. - // If max_buffers is -1, no memory limit will apply. - static int64_t calculate_mem_tracker(int max_buffers, int block_size); - - ExecEnv* exec_env() { return _exec_env; } - -private: - ExecEnv* _exec_env; - - // Per-query states with associated block managers. - std::vector > _query_states; - - StorageEngine* _engine = nullptr; -}; - -} // end namespace doris - -#endif // DORIS_BE_TEST_QUERY_RUNTIME_TEST_ENV_H diff --git a/be/test/runtime/user_function_cache_test.cpp b/be/test/runtime/user_function_cache_test.cpp index be1705ba9abed61..b9a4694982d6b40 100644 --- a/be/test/runtime/user_function_cache_test.cpp +++ b/be/test/runtime/user_function_cache_test.cpp @@ -16,3 +16,31 @@ // under the License. #include "runtime/user_function_cache.h" + +#include +#include + +#include + +#include "gtest/gtest.h" + +namespace doris { + +class UserFunctionCacheTest : public ::testing::Test { +protected: + UserFunctionCache ufc; +}; + +TEST_F(UserFunctionCacheTest, SplitStringByChecksumTest) { + // Test valid string format + std::string valid_str = + "7119053928154065546.20c8228267b6c9ce620fddb39467d3eb.postgresql-42.5.0.jar"; + auto result = ufc._split_string_by_checksum(valid_str); + ASSERT_EQ(result.size(), 4); + EXPECT_EQ(result[0], "7119053928154065546"); + EXPECT_EQ(result[1], "20c8228267b6c9ce620fddb39467d3eb"); + EXPECT_EQ(result[2], "postgresql-42.5.0"); + EXPECT_EQ(result[3], "jar"); +} + +} // namespace doris diff --git a/be/test/testutil/run_all_tests.cpp b/be/test/testutil/run_all_tests.cpp index ab3b0b1ea026c6e..2e735ce8fa47b3a 100644 --- a/be/test/testutil/run_all_tests.cpp +++ b/be/test/testutil/run_all_tests.cpp @@ -28,6 +28,7 @@ #include "olap/segment_loader.h" #include "olap/tablet_schema_cache.h" #include "runtime/exec_env.h" +#include "runtime/memory/cache_manager.h" #include "runtime/memory/thread_mem_tracker_mgr.h" #include "runtime/thread_context.h" #include "service/backend_options.h" diff --git a/be/test/vec/columns/column_fixed_length_object_test.cpp b/be/test/vec/columns/column_fixed_length_object_test.cpp index 1f4b2d6d7f62522..fd0a3687d659573 100644 --- a/be/test/vec/columns/column_fixed_length_object_test.cpp +++ b/be/test/vec/columns/column_fixed_length_object_test.cpp @@ -57,18 +57,20 @@ TEST(ColumnFixedLenghtObjectTest, InsertRangeFrom) { } TEST(ColumnFixedLenghtObjectTest, UpdateHashWithValue) { - auto column1 = ColumnFixedLengthObject::create(sizeof(size_t)); - EXPECT_EQ(sizeof(size_t), column1->item_size()); + auto column1 = ColumnFixedLengthObject::create(sizeof(int64_t)); + EXPECT_EQ(sizeof(int64_t), column1->item_size()); const size_t count = 1000; column1->resize(count); auto& data = column1->get_data(); - for (size_t i = 0; i < count; ++i) { - *((size_t*)&data[i * sizeof(size_t)]) = i; + for (size_t i = 0; i != count; ++i) { + *((int64_t*)&data[i * column1->item_size()]) = i; } SipHash hash1; - column1->update_hash_with_value(count, hash1); + for (size_t i = 0; i != count; ++i) { + column1->update_hash_with_value(i, hash1); + } auto column2 = ColumnVector::create(); column2->resize(count); @@ -77,7 +79,9 @@ TEST(ColumnFixedLenghtObjectTest, UpdateHashWithValue) { } SipHash hash2; - column2->update_hash_with_value(count, hash2); + for (size_t i = 0; i != count; ++i) { + column2->update_hash_with_value(i, hash2); + } EXPECT_EQ(hash1.get64(), hash2.get64()); } diff --git a/be/test/vec/core/block_test.cpp b/be/test/vec/core/block_test.cpp index 456d4fc4807c8ea..61903e588f2fba3 100644 --- a/be/test/vec/core/block_test.cpp +++ b/be/test/vec/core/block_test.cpp @@ -129,7 +129,8 @@ void serialize_and_deserialize_test(segment_v2::CompressionTypePB compression_ty block_to_pb(block, &pblock, compression_type); std::string s1 = pblock.DebugString(); - vectorized::Block block2(pblock); + vectorized::Block block2; + block2.deserialize(pblock); PBlock pblock2; block_to_pb(block2, &pblock2, compression_type); std::string s2 = pblock2.DebugString(); @@ -150,7 +151,8 @@ void serialize_and_deserialize_test(segment_v2::CompressionTypePB compression_ty block_to_pb(block, &pblock, compression_type); std::string s1 = pblock.DebugString(); - vectorized::Block block2(pblock); + vectorized::Block block2; + block2.deserialize(pblock); PBlock pblock2; block_to_pb(block2, &pblock2, compression_type); std::string s2 = pblock2.DebugString(); @@ -174,7 +176,8 @@ void serialize_and_deserialize_test(segment_v2::CompressionTypePB compression_ty block_to_pb(block, &pblock, compression_type); std::string s1 = pblock.DebugString(); - vectorized::Block block2(pblock); + vectorized::Block block2; + block2.deserialize(pblock); PBlock pblock2; block_to_pb(block2, &pblock2, compression_type); std::string s2 = pblock2.DebugString(); @@ -200,7 +203,8 @@ void serialize_and_deserialize_test(segment_v2::CompressionTypePB compression_ty block_to_pb(block, &pblock, compression_type); std::string s1 = pblock.DebugString(); - vectorized::Block block2(pblock); + vectorized::Block block2; + block2.deserialize(pblock); PBlock pblock2; block_to_pb(block2, &pblock2, compression_type); std::string s2 = pblock2.DebugString(); @@ -220,7 +224,8 @@ void serialize_and_deserialize_test(segment_v2::CompressionTypePB compression_ty block_to_pb(block, &pblock, compression_type); std::string s1 = pblock.DebugString(); - vectorized::Block block2(pblock); + vectorized::Block block2; + block2.deserialize(pblock); PBlock pblock2; block_to_pb(block2, &pblock2, compression_type); std::string s2 = pblock2.DebugString(); @@ -242,7 +247,8 @@ void serialize_and_deserialize_test(segment_v2::CompressionTypePB compression_ty EXPECT_TRUE(pblock.column_metas()[0].has_decimal_param()); std::string s1 = pblock.DebugString(); - vectorized::Block block2(pblock); + vectorized::Block block2; + block2.deserialize(pblock); PBlock pblock2; block_to_pb(block2, &pblock2, compression_type); std::string s2 = pblock2.DebugString(); @@ -264,7 +270,8 @@ void serialize_and_deserialize_test(segment_v2::CompressionTypePB compression_ty block_to_pb(block, &pblock, compression_type); std::string s1 = pblock.DebugString(); - vectorized::Block block2(pblock); + vectorized::Block block2; + block2.deserialize(pblock); PBlock pblock2; block_to_pb(block2, &pblock2, compression_type); std::string s2 = pblock2.DebugString(); @@ -279,7 +286,8 @@ void serialize_and_deserialize_test(segment_v2::CompressionTypePB compression_ty block_to_pb(block, &pblock, compression_type); std::string s1 = pblock.DebugString(); - vectorized::Block block2(pblock); + vectorized::Block block2; + block2.deserialize(pblock); PBlock pblock2; block_to_pb(block2, &pblock2, compression_type); std::string s2 = pblock2.DebugString(); diff --git a/be/test/vec/exec/vtablet_sink_test.cpp b/be/test/vec/exec/vtablet_sink_test.cpp index 5e60181c8fb80ea..75ed927fcb2acc1 100644 --- a/be/test/vec/exec/vtablet_sink_test.cpp +++ b/be/test/vec/exec/vtablet_sink_test.cpp @@ -316,7 +316,8 @@ class VTestInternalService : public PBackendService { k_add_batch_status.to_protobuf(response->mutable_status()); if (request->has_block() && _row_desc != nullptr) { - vectorized::Block block(request->block()); + vectorized::Block block; + block.deserialize(request->block()); for (size_t row_num = 0; row_num < block.rows(); ++row_num) { std::stringstream out; diff --git a/be/test/vec/function/function_test_util.cpp b/be/test/vec/function/function_test_util.cpp index f75be089104016d..c61d272cca8266a 100644 --- a/be/test/vec/function/function_test_util.cpp +++ b/be/test/vec/function/function_test_util.cpp @@ -23,6 +23,7 @@ #include #include "runtime/jsonb_value.h" +#include "runtime/runtime_state.h" #include "util/binary_cast.hpp" #include "util/bitmap_value.h" #include "vec/data_types/data_type_array.h" @@ -341,8 +342,9 @@ Block* process_table_function(TableFunction* fn, Block* input_block, return nullptr; } + RuntimeState runtime_state((TQueryGlobals())); // process table function init - if (fn->process_init(input_block) != Status::OK()) { + if (fn->process_init(input_block, &runtime_state) != Status::OK()) { LOG(WARNING) << "TableFunction process_init failed"; return nullptr; } diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/data_case/partial_update/update.csv b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/data_case/partial_update/update.csv new file mode 100644 index 000000000000000..7abdf2f85d8b3c9 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/data_case/partial_update/update.csv @@ -0,0 +1,2 @@ +2,400 +1,200 \ No newline at end of file diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/data_case/partial_update/update2.csv b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/data_case/partial_update/update2.csv new file mode 100644 index 000000000000000..f7f6aabd57d307e --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/data_case/partial_update/update2.csv @@ -0,0 +1,2 @@ +1,999 +3,888 diff --git a/docker/thirdparties/docker-compose/sqlserver/init/03-create-table.sql b/docker/thirdparties/docker-compose/sqlserver/init/03-create-table.sql index 09b1cd06e79aca9..a491647e785ec05 100644 --- a/docker/thirdparties/docker-compose/sqlserver/init/03-create-table.sql +++ b/docker/thirdparties/docker-compose/sqlserver/init/03-create-table.sql @@ -96,4 +96,8 @@ CREATE TABLE dbo.DateAndTime DateTimeOffsetColumn DATETIMEOFFSET ); +CREATE TABLE dbo.t_id ( + ID uniqueidentifier PRIMARY KEY, + Name nvarchar(100) +); diff --git a/docker/thirdparties/docker-compose/sqlserver/init/04-insert.sql b/docker/thirdparties/docker-compose/sqlserver/init/04-insert.sql index d912c18fcc4da3c..f4f67523dfb6077 100644 --- a/docker/thirdparties/docker-compose/sqlserver/init/04-insert.sql +++ b/docker/thirdparties/docker-compose/sqlserver/init/04-insert.sql @@ -60,3 +60,6 @@ VALUES ( '2023-06-25 14:30:45.1234567 -07:00' -- DATETIMEOFFSET ); +INSERT INTO dbo.t_id (ID, Name) VALUES (NEWID(), 'Data 1'); +INSERT INTO dbo.t_id (ID, Name) VALUES (NEWID(), 'Data 2'); + diff --git a/docs/en/docs/admin-manual/maint-monitor/automatic-service-start.md b/docs/en/docs/admin-manual/maint-monitor/automatic-service-start.md index 326692b76b9f93f..cc8db14c863743b 100644 --- a/docs/en/docs/admin-manual/maint-monitor/automatic-service-start.md +++ b/docs/en/docs/admin-manual/maint-monitor/automatic-service-start.md @@ -49,9 +49,14 @@ doris ALL=(ALL) NOPASSWD:DORISCTL ### Configuration procedure -1. Download the doris-fe.service file: [doris-fe.service](https://github.com/apache/doris/blob/master/tools/systemd/doris-fe.service) +1. You should config the "JAVA_HOME" variable in the config file, both fe.conf and be.conf, or you can't use the command "systemctl start" to start doris + ``` + echo "JAVA_HOME=your_java_home" >> /home/doris/fe/conf/fe.conf + echo "JAVA_HOME=your_java_home" >> /home/doris/be/conf/be.conf + ``` +2. Download the doris-fe.service file: [doris-fe.service](https://github.com/apache/doris/blob/master/tools/systemd/doris-fe.service) -2. The details of doris-fe.service are as follows: +3. The details of doris-fe.service are as follows: ``` # Licensed to the Apache Software Foundation (ASF) under one @@ -98,9 +103,9 @@ doris ALL=(ALL) NOPASSWD:DORISCTL - ExecStart and ExecStop are configured based on actual fe paths -3. Download the doris-be.service file : [doris-be.service](https://github.com/apache/doris/blob/master/tools/systemd/doris-be.service) +4. Download the doris-be.service file : [doris-be.service](https://github.com/apache/doris/blob/master/tools/systemd/doris-be.service) -4. The details of doris-be.service are as follows: +5. The details of doris-be.service are as follows: ``` # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -146,11 +151,11 @@ doris ALL=(ALL) NOPASSWD:DORISCTL - ExecStart and ExecStop are configured based on actual be paths -5. Service configuration +6. Service configuration Place doris-fe.service and doris-be.service in the /usr/lib/systemd/system directory -6. Set self-start +7. Set self-start After you add or modify the configuration file, you need to reload it @@ -165,7 +170,7 @@ doris ALL=(ALL) NOPASSWD:DORISCTL systemctl enable doris-be ``` -7. Service initiation +8. Service initiation ``` systemctl start doris-fe diff --git a/docs/en/docs/advanced/cold-hot-separation.md b/docs/en/docs/advanced/cold-hot-separation.md index 7b954c577b40a48..f98fdf0a804b15d 100644 --- a/docs/en/docs/advanced/cold-hot-separation.md +++ b/docs/en/docs/advanced/cold-hot-separation.md @@ -102,6 +102,7 @@ Or associate a storage policy with an existing partition ``` ALTER TABLE create_table_partition MODIFY PARTITION (*) SET("storage_policy"="test_policy"); ``` +**Note**: If the user specifies different storage policies for the entire table and certain partitions during table creation, the storage policy set for the partitions will be ignored, and all partitions of the table will use the table's policy. If you need a specific partition to have a different policy than the others, you can modify it by associating the partition with the desired storage policy, as mentioned earlier in the context of modifying an existing partition. For details, please refer to the [resource](../sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-RESOURCE.md), [policy](../sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-POLICY.md), create table, alter and other documents in the docs directory ### Some restrictions diff --git a/docs/en/docs/advanced/variables.md b/docs/en/docs/advanced/variables.md index 97dee5752c868e0..f2838dde2304973 100644 --- a/docs/en/docs/advanced/variables.md +++ b/docs/en/docs/advanced/variables.md @@ -390,7 +390,7 @@ Translated with www.DeepL.com/Translator (free version) * `sql_select_limit` - Used for compatibility with MySQL clients. No practical effect. + Used to limit return rows of select stmt, including select clause of insert stmt. * `system_time_zone` diff --git a/docs/en/docs/data-operate/import/import-way/stream-load-manual.md b/docs/en/docs/data-operate/import/import-way/stream-load-manual.md index f18bccc435029b3..58f90de98b59c7d 100644 --- a/docs/en/docs/data-operate/import/import-way/stream-load-manual.md +++ b/docs/en/docs/data-operate/import/import-way/stream-load-manual.md @@ -245,16 +245,13 @@ Stream load uses HTTP protocol, so all parameters related to import tasks are se You can add a `sql` parameter to the `Header` to replace the `column_separator`, `line_delimiter`, `where`, `columns` in the previous parameter, which is convenient to use. ``` -curl --location-trusted -u user:passwd -[-H "sql: ${load_sql}"...] --T data.file --XPUT http://fe_host:http_port/api/{db}/{table}/_stream_load_with_sql +curl --location-trusted -u user:passwd [-H "sql: ${load_sql}"...] -T data.file -XPUT http://fe_host:http_port/api/_http_stream # -- load_sql -# insert into db.table (col, ...) select stream_col, ... from stream("property1"="value1"); +# insert into db.table (col, ...) select stream_col, ... from http_stream("property1"="value1"); -# stream +# http_stream # ( # "column_separator" = ",", # "format" = "CSV", @@ -265,7 +262,7 @@ curl --location-trusted -u user:passwd Examples: ``` -curl --location-trusted -u root: -T test.csv -H "sql:insert into demo.example_tbl_1(user_id, age, cost) select c1, c4, c7 * 2 from stream("format" = "CSV", "column_separator" = "," ) where age >= 30" http://127.0.0.1:28030/api/demo/example_tbl_1/_stream_load_with_sql +curl --location-trusted -u root: -T test.csv -H "sql:insert into demo.example_tbl_1(user_id, age, cost) select c1, c4, c7 * 2 from http_stream("format" = "CSV", "column_separator" = "," ) where age >= 30" http://127.0.0.1:28030/api/_http_stream ``` ### Return results diff --git a/docs/en/docs/data-table/index/inverted-index.md b/docs/en/docs/data-table/index/inverted-index.md index e3ee40c64bea16d..c711b1fca47222c 100644 --- a/docs/en/docs/data-table/index/inverted-index.md +++ b/docs/en/docs/data-table/index/inverted-index.md @@ -168,6 +168,51 @@ SELECT * FROM table_name WHERE ts > '2023-01-01 00:00:00'; SELECT * FROM table_name WHERE op_type IN ('add', 'delete'); ``` +- Tokenization Function + +To evaluate the actual effects of tokenization or to tokenize a block of text, the `tokenize` function can be utilized. +```sql +mysql> SELECT TOKENIZE('武汉长江大桥','"parser"="chinese","parser_mode"="fine_grained"); ++-----------------------------------------------------------------------------------+ +| tokenize('武汉长江大桥', '"parser"="chinese","parser_mode"="fine_grained"') | ++-----------------------------------------------------------------------------------+ +| ["武汉", "武汉长江大桥", "长江", "长江大桥", "大桥"] | ++-----------------------------------------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> SELECT TOKENIZE('武汉市长江大桥','"parser"="chinese","parser_mode"="fine_grained"); ++--------------------------------------------------------------------------------------+ +| tokenize('武汉市长江大桥', '"parser"="chinese","parser_mode"="fine_grained"') | ++--------------------------------------------------------------------------------------+ +| ["武汉", "武汉市", "市长", "长江", "长江大桥", "大桥"] | ++--------------------------------------------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> SELECT TOKENIZE('武汉市长江大桥','"parser"="chinese","parser_mode"="coarse_grained"); ++----------------------------------------------------------------------------------------+ +| tokenize('武汉市长江大桥', '"parser"="chinese","parser_mode"="coarse_grained"') | ++----------------------------------------------------------------------------------------+ +| ["武汉市", "长江大桥"] | ++----------------------------------------------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> SELECT TOKENIZE('I love CHINA','"parser"="english"); ++------------------------------------------------+ +| tokenize('I love CHINA', '"parser"="english"') | ++------------------------------------------------+ +| ["i", "love", "china"] | ++------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> SELECT TOKENIZE('I love CHINA 我爱我的祖国','"parser"="unicode"); ++-------------------------------------------------------------------+ +| tokenize('I love CHINA 我爱我的祖国', '"parser"="unicode"') | ++-------------------------------------------------------------------+ +| ["i", "love", "china", "我", "爱", "我", "的", "祖", "国"] | ++-------------------------------------------------------------------+ +1 row in set (0.02 sec) +``` + ## Examples This example will demostrate inverted index creation, fulltext query, normal query using a hackernews dataset with 1 million rows. The performanc comparation between using and without inverted index will also be showed. diff --git a/docs/en/docs/ecosystem/flink-doris-connector.md b/docs/en/docs/ecosystem/flink-doris-connector.md index 5b7620c6594e8c5..dbb0e9d99d33b07 100644 --- a/docs/en/docs/ecosystem/flink-doris-connector.md +++ b/docs/en/docs/ecosystem/flink-doris-connector.md @@ -89,7 +89,7 @@ CREATE TABLE flink_doris_source ( ) WITH ( 'connector' = 'doris', - 'fenodes' = 'FE_IP:8030', + 'fenodes' = 'FE_IP:HTTP_PORT', 'table.identifier' = 'database.table', 'username' = 'root', 'password' = 'password' @@ -100,7 +100,7 @@ CREATE TABLE flink_doris_source ( ```java DorisOptions.Builder builder = DorisOptions.builder() - .setFenodes("FE_IP:8030") + .setFenodes("FE_IP:HTTP_PORT") .setTableIdentifier("db.table") .setUsername("root") .setPassword("password"); @@ -131,7 +131,7 @@ CREATE TABLE flink_doris_sink ( ) WITH ( 'connector' = 'doris', - 'fenodes' = 'FE_IP:8030', + 'fenodes' = 'FE_IP:HTTP_PORT', 'table.identifier' = 'db.table', 'username' = 'root', 'password' = 'password', @@ -156,7 +156,7 @@ env.setRuntimeMode(RuntimeExecutionMode.BATCH); DorisSink.Builder builder = DorisSink.builder(); DorisOptions.Builder dorisBuilder = DorisOptions.builder(); -dorisBuilder.setFenodes("FE_IP:8030") +dorisBuilder.setFenodes("FE_IP:HTTP_PORT") .setTableIdentifier("db.table") .setUsername("root") .setPassword("password"); @@ -190,7 +190,7 @@ env.setRuntimeMode(RuntimeExecutionMode.BATCH); //doris sink option DorisSink.Builder builder = DorisSink.builder(); DorisOptions.Builder dorisBuilder = DorisOptions.builder(); -dorisBuilder.setFenodes("FE_IP:8030") +dorisBuilder.setFenodes("FE_IP:HTTP_PORT") .setTableIdentifier("db.table") .setUsername("root") .setPassword("password"); @@ -301,15 +301,16 @@ ON a.city = c.city ### General configuration items -| Key | Default Value | Required | Comment | -| -------------------------------- | ------------- | -------- | ------------------------------------------------------------ | -| fenodes | -- | Y | Doris FE http address, multiple addresses are supported, separated by commas | -| table.identifier | -- | Y | Doris table name, such as: db.tbl | -| username | -- | Y | username to access Doris | -| password | -- | Y | Password to access Doris | -| doris.request.retries | 3 | N | Number of retries to send requests to Doris | -| doris.request.connect.timeout.ms | 30000 | N | Connection timeout for sending requests to Doris | -| doris.request.read.timeout.ms | 30000 | N | Read timeout for sending requests to Doris | +| Key | Default Value | Required | Comment | +|----------------------------------|---------------|----------|---------------------------------------------------------------------------------------------------------------------------------------------------------| +| fenodes | -- | Y | Doris FE http address, multiple addresses are supported, separated by commas | +| benodes | -- | N | Doris BE http address, multiple addresses are supported, separated by commas. refer to [#187](https://github.com/apache/doris-flink-connector/pull/187) | +| table.identifier | -- | Y | Doris table name, such as: db.tbl | +| username | -- | Y | username to access Doris | +| password | -- | Y | Password to access Doris | +| doris.request.retries | 3 | N | Number of retries to send requests to Doris | +| doris.request.connect.timeout.ms | 30000 | N | Connection timeout for sending requests to Doris | +| doris.request.read.timeout.ms | 30000 | N | Read timeout for sending requests to Doris | ### Source configuration item @@ -439,6 +440,8 @@ insert into doris_sink select id,name from cdc_mysql_source; - **--oracle-conf** Oracle CDCSource configuration, for example --oracle-conf hostname=127.0.0.1, you can view all configurations of Oracle-CDC in [here](https://ververica.github.io/flink-cdc-connectors/master/content/connectors/oracle-cdc.html), where hostname/username/password/database-name/schema-name is required. - **--sink-conf** All configurations of Doris Sink, you can view the complete configuration items in [here](https://doris.apache.org/zh-CN/docs/dev/ecosystem/flink-doris-connector/#%E9%80%9A%E7%94%A8%E9%85%8D%E7%BD%AE%E9%A1%B9). - **--table-conf** The configuration item of the Doris table, that is, the content contained in properties. For example --table-conf replication_num=1 +- **--ignore-default-value** Turn off the default for synchronizing mysql table structures. It is suitable for synchronizing mysql data to doris, the field has a default value, but the actual inserted data is null. refer to[#152](https://github.com/apache/doris-flink-connector/pull/152) +- **--use-new-schema-change** The new schema change supports synchronous mysql multi-column changes and default values. refer to[#167](https://github.com/apache/doris-flink-connector/pull/167) >Note: When synchronizing, you need to add the corresponding Flink CDC dependencies in the $FLINK_HOME/lib directory, such as flink-sql-connector-mysql-cdc-${version}.jar, flink-sql-connector-oracle-cdc-${version}.jar diff --git a/docs/en/docs/ecosystem/udf/java-user-defined-function.md b/docs/en/docs/ecosystem/udf/java-user-defined-function.md index c6b57fed3cb3455..d1e3ee29984a64f 100644 --- a/docs/en/docs/ecosystem/udf/java-user-defined-function.md +++ b/docs/en/docs/ecosystem/udf/java-user-defined-function.md @@ -59,8 +59,9 @@ Java UDF provides users with a Java interface written in UDF to facilitate the e |String|String| |Decimal|BigDecimal| |```array```|```ArrayList```| +|```map```|```HashMap```| -* Array types can nested basic types, Eg: In Doris: ```array``` corresponds to JAVA UDF Argument Type: ```ArrayList```, Others is also. +* Array/Map types can nested basic types, Eg: In Doris: ```array``` corresponds to JAVA UDF Argument Type: ```ArrayList```, Others is also. ## Write UDF functions This section mainly introduces how to develop a Java UDF. Samples for the Java version are provided under `samples/doris-demo/java-udf-demo/` for your reference, Check it out [here](https://github.com/apache/incubator-doris/tree/master/samples/doris-demo/java-udf-demo) diff --git a/docs/en/docs/sql-manual/sql-functions/date-time-functions/date_ceil.md b/docs/en/docs/sql-manual/sql-functions/date-time-functions/date_ceil.md new file mode 100644 index 000000000000000..d2a988295ba7aca --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/date-time-functions/date_ceil.md @@ -0,0 +1,96 @@ +--- +{ + "title": "date_ceil", + "language": "en" +} +--- + + + +## date_ceil +### description +#### Syntax + +`DATETIME DATE_CEIL(DATETIME datetime, INTERVAL period type)` + + +Convert the date to the nearest rounding up time of the specified time interval period. + +The datetime parameter is a valid date expression. + +The period parameter specifies how many units each cycle consists of, starting from 0001-01-01T00:00:00 + +type :YEAR, MONTH, DAY, HOUR, MINUTE, SECOND. + +### example + +``` +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 second); ++--------------------------------------------------------------+ +| second_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++--------------------------------------------------------------+ +| 2023-07-13 22:28:20 | ++--------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 minute); ++--------------------------------------------------------------+ +| minute_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++--------------------------------------------------------------+ +| 2023-07-13 22:30:00 | ++--------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 hour); ++------------------------------------------------------------+ +| hour_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++------------------------------------------------------------+ +| 2023-07-13 23:00:00 | ++------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 day); ++-----------------------------------------------------------+ +| day_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++-----------------------------------------------------------+ +| 2023-07-15 00:00:00 | ++-----------------------------------------------------------+ +1 row in set (0.00 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 month); ++-------------------------------------------------------------+ +| month_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++-------------------------------------------------------------+ +| 2023-12-01 00:00:00 | ++-------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 year); ++------------------------------------------------------------+ +| year_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++------------------------------------------------------------+ +| 2026-01-01 00:00:00 | ++------------------------------------------------------------+ +1 row in set (0.00 sec) +``` + +### keywords + + DATE_CEIL,DATE,CEIL diff --git a/docs/en/docs/sql-manual/sql-functions/date-time-functions/date_floor.md b/docs/en/docs/sql-manual/sql-functions/date-time-functions/date_floor.md new file mode 100644 index 000000000000000..cc2ac666d95e256 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/date-time-functions/date_floor.md @@ -0,0 +1,104 @@ +--- +{ + "title": "date_floor", + "language": "en" +} +--- + + + +## date_floor +### description +#### Syntax + +`DATETIME DATE_FLOOR(DATETIME datetime, INTERVAL period type)` + + +Converts a date to the nearest rounding down time of a specified time interval period. + +The datetime parameter is a valid date expression. + +The period parameter specifies how many units each cycle consists of, starting from 0001-01-01T00:00:00 + +type :YEAR, MONTH, DAY, HOUR, MINUTE, SECOND. + +### example + +``` +mysql>select date_floor("0001-01-01 00:00:16",interval 5 second); ++---------------------------------------------------------------+ +| second_floor('0001-01-01 00:00:16', 5, '0001-01-01 00:00:00') | ++---------------------------------------------------------------+ +| 0001-01-01 00:00:15 | ++---------------------------------------------------------------+ +1 row in set (0.00 sec) + +mysql>select date_floor("0001-01-01 00:00:18",interval 5 second); ++---------------------------------------------------------------+ +| second_floor('0001-01-01 00:00:18', 5, '0001-01-01 00:00:00') | ++---------------------------------------------------------------+ +| 0001-01-01 00:00:15 | ++---------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 minute); ++---------------------------------------------------------------+ +| minute_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++---------------------------------------------------------------+ +| 2023-07-13 22:25:00 | ++---------------------------------------------------------------+ +1 row in set (0.00 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 hour); ++-------------------------------------------------------------+ +| hour_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++-------------------------------------------------------------+ +| 2023-07-13 18:00:00 | ++-------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 day); ++------------------------------------------------------------+ +| day_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++------------------------------------------------------------+ +| 2023-07-10 00:00:00 | ++------------------------------------------------------------+ +1 row in set (0.00 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 month); ++--------------------------------------------------------------+ +| month_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++--------------------------------------------------------------+ +| 2023-07-01 00:00:00 | ++--------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 year); ++-------------------------------------------------------------+ +| year_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++-------------------------------------------------------------+ +| 2021-01-01 00:00:00 | ++-------------------------------------------------------------+ + +``` + +### keywords + + DATE_FLOOR,DATE,FLOOR diff --git a/docs/en/docs/sql-manual/sql-functions/table-functions/hdfs.md b/docs/en/docs/sql-manual/sql-functions/table-functions/hdfs.md index 5969585ad583f58..7cbd21366a207b5 100644 --- a/docs/en/docs/sql-manual/sql-functions/table-functions/hdfs.md +++ b/docs/en/docs/sql-manual/sql-functions/table-functions/hdfs.md @@ -69,7 +69,7 @@ Related parameters for accessing HDFS in HA mode: File format parameters: -- `format`: (required) Currently support `csv/csv_with_names/csv_with_names_and_types/json/parquet/orc` +- `format`: (required) Currently support `csv/csv_with_names/csv_with_names_and_types/json/parquet/orc/avro` - `column_separator`: (optional) default `,`. - `line_delimiter`: (optional) default `\n`. - `compress_type`: (optional) Currently support `UNKNOWN/PLAIN/GZ/LZO/BZ2/LZ4FRAME/DEFLATE`. Default value is `UNKNOWN`, it will automatically infer the type based on the suffix of `uri`. diff --git a/docs/en/docs/sql-manual/sql-functions/table-functions/s3.md b/docs/en/docs/sql-manual/sql-functions/table-functions/s3.md index b42788583fcf423..d089c981558bcc7 100644 --- a/docs/en/docs/sql-manual/sql-functions/table-functions/s3.md +++ b/docs/en/docs/sql-manual/sql-functions/table-functions/s3.md @@ -424,6 +424,25 @@ MySQL [(none)]> select * from s3( +-----------+------------------------------------------+----------------+----------+-------------------------+--------+-------------+---------------+---------------------+ ``` +**avro format** + +`avro` format: S3 tvf supports parsing the column names and column types of the table schema from the avro file. Example: + +```sql +select * from s3( + "uri" = "http://127.0.0.1:9312/test2/person.avro", + "ACCESS_KEY" = "ak", + "SECRET_KEY" = "sk", + "FORMAT" = "avro"); ++--------+--------------+-------------+-----------------+ +| name | boolean_type | double_type | long_type | ++--------+--------------+-------------+-----------------+ +| Alyssa | 1 | 10.0012 | 100000000221133 | +| Ben | 0 | 5555.999 | 4009990000 | +| lisi | 0 | 5992225.999 | 9099933330 | ++--------+--------------+-------------+-----------------+ +``` + **uri contains wildcards** uri can use wildcards to read multiple files. Note: If wildcards are used, the format of each file must be consistent (especially csv/csv_with_names/csv_with_names_and_types count as different formats), S3 tvf uses the first file to parse out the table schema. For example: diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md index d1774fcf4778d86..03aa6968eae1859 100644 --- a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md +++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md @@ -64,8 +64,8 @@ Notice: - If the bucketing method is specified, only the number of buckets can be modified, not the bucketing method or the bucketing column. If the bucketing method is specified but the number of buckets not be specified, the default value `10` will be used for bucket number instead of the number specified when the table is created. If the number of buckets modified, the bucketing method needs to be specified simultaneously. - The ["key"="value"] section can set some attributes of the partition, see [CREATE TABLE](../Create/CREATE-TABLE.md) - If the user does not explicitly create a partition when creating a table, adding a partition by ALTER is not supported -- If the user uses list partition then they can add default partition to the table - - ALTER TABLE ADD PARTITION DEFAULT +- If the user uses list partition then they can add default partition to the table. The default partition will store all data not satisfying prior partition key's constraints. + - ALTER TABLE table_name ADD PARTITION partition_name 2. Delete the partition diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/PAUSE-JOB.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/PAUSE-JOB.md new file mode 100644 index 000000000000000..8f8dcbce6d4b1f3 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/PAUSE-JOB.md @@ -0,0 +1,53 @@ +--- +{ +"title": "PAUSE-JOB", +"language": "en" +} +--- + + + +## PAUSE-JOB + +### Name + +PAUSE JOB + +### Description + +User suspends a JOB job. A stopped job can be resumed with RESUME JOB. + +```sql +PAUSE JOB FOR job_name; +``` + +### Example + +1. Pause the job named test1. + + ```sql + PAUSE JOB FOR test1; + ``` + +###Keywords + + PAUSE, JOB + +### Best Practice diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/RESUME-JOB.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/RESUME-JOB.md new file mode 100644 index 000000000000000..49eb11b23016ce4 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/RESUME-JOB.md @@ -0,0 +1,54 @@ +--- +{ +"title": "RESUME-JOB", +"language": "en" +} +--- + + + +## RESUME-JOB + +### Name + +RESUME JOB + +### Description + +Used to restart a JOB in PAUSE state. The restarted job will continue to be executed periodically. A JOB in STOP state cannot be resumed. + +```sql +RESUME JOB FOR job_name; +``` + +### Example + +1. Restart the JOB named test1. + + ```sql + RESUME JOB FOR test1; + ``` + +### Keywords + + RESUME, JOB + +### Best Practice + diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/STOP-JOB.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/STOP-JOB.md new file mode 100644 index 000000000000000..05b9d08111414e2 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/STOP-JOB.md @@ -0,0 +1,54 @@ +--- +{ +"title": "STOP-JOB", +"language": "en" +} +--- + + + +## STOP-JOB + +### Name + +STOP JOB + +### Description + +User stops a JOB job. A stopped job cannot be rerun. + +```sql +STOP JOB FOR job_name; +``` + +### Example + +1. Stop the job named test1. + + ```sql + STOP JOB FOR test1; + ``` + +### Keywords + + STOP, JOB + +### Best Practice + diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-JOB.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-JOB.md new file mode 100644 index 000000000000000..d89e0aad0d2aea0 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-JOB.md @@ -0,0 +1,129 @@ +--- +{ +"title": "CREATE-JOB", +"language": "en" +} + +--- + + +## CREATE-JOB + +### Name + +CREATE JOB + +### Description + +Doris Job is a task that runs according to a predetermined plan and is used to trigger predefined actions at a specific time or at a specified time interval, thereby helping us to automate some tasks. Functionally, it is similar to the operating system's +Timing tasks (such as: cron in Linux, scheduled tasks in Windows). But Doris's job scheduling can be accurate to the second level. + +There are two types of jobs: `ONE_TIME` and `BATCH`. Among them, the `ONE_TIME` type Job will be triggered at the specified time point, which is mainly used for one-time tasks, while the `BATCH` type Job will be triggered cyclically within the specified time interval. +Mainly used for tasks that are executed periodically. + +Currently only ***ADMIN*** permissions are supported for this operation. + +grammar: + +```sql +CREATE + job + job_name + ON SCHEDULE schedule + [COMMENT 'string'] + DO sql_body; + +schedule: { + AT timestamp + | EVERY interval + [STARTS timestamp] + [ENDS timestamp ] +} + +interval: + quantity { DAY | HOUR | MINUTE | + WEEK | SECOND } +``` + +A valid Job statement must contain the following + +- The keyword CREATE JOB plus the job name, which identifies unique events within a db. +- The ON SCHEDULE clause, which specifies the type of Job and when and how often to trigger it. +- The DO clause, which specifies the actions that need to be performed when the Job job is triggered. + +Here is a minimal example: + +```sql +CREATE JOB my_job ON SCHEDULE EVERY 1 MINUTE DO INSERT INTO db1.tbl1 SELECT * FROM db2.tbl2; +``` + +This statement means to create a job named my_job to be executed every minute, and the operation performed is to import the data in db2.tbl2 into db1.tbl1. + +The SCHEDULER statement is used to define the execution time, frequency and duration of the job, which can specify a one-time job or a periodic job. +- AT timestamp + + For one-time events, it specifies that the event is only executed once at a given date and time timestamp, which must contain the date and time + +- EVERY + + Indicates that the operation is repeated periodically, which specifies the execution frequency of the job. After the keyword, a time interval should be specified, which can be days, hours, minutes, seconds, and weeks. + + - interval + + Used to specify the job execution frequency, it can be days, hours, minutes, seconds, weeks. For example: `1 DAY` means execute once a day, `1 HOUR` means execute once an hour, `1 MINUTE` means execute once every minute, `1 WEEK` means execute once a week, `1 SECOND` means execute once every second . + + - STARTS timestamp + + It is used to specify the start time of the job. If not specified, it will be executed from the next time point of the current time. + + - ENDS timestamp + + Used to specify the end time of the job, if not specified, it means permanent execution. +- DO + + It is used to specify the operation that needs to be performed when the job is triggered. Currently, all ***INSERT, UPDATE*** operations are supported. We will support more operations in the future. + +### Example + +Create a one-time job, which will be executed once at 2020-01-01 00:00:00, and the operation performed is to import the data in db2.tbl2 into db1.tbl1. + +```sql + +CREATE JOB my_job ON SCHEDULE AT '2020-01-01 00:00:00' DO INSERT INTO db1.tbl1 SELECT * FROM db2.tbl2; + +``` + +Create a periodic Job, which will start to execute at 2020-01-01 00:00:00, once a day, and the operation is to import the data in db2.tbl2 into db1.tbl1. + +```sql +CREATE JOB my_job ON SCHEDULE EVERY 1 DAY STARTS '2020-01-01 00:00:00' DO INSERT INTO db1.tbl1 SELECT * FROM db2.tbl2 WHERE create_time >= days_add(now(),-1); +``` + +Create a periodic Job, which will start to execute at 2020-01-01 00:00:00, and execute once a day. The operation performed is to import the data in db2.tbl2 into db1.tbl1. This Job will be executed in 2020 Ends at -01-01 00:10:00. + +```sql +CREATE JOB my_job ON SCHEDULER EVERY 1 DAY STARTS '2020-01-01 00:00:00' ENDS '2020-01-01 00:10:00' DO INSERT INTO db1.tbl1 SELECT * FROM db2.tbl2 create_time >= days_add (now(),-1); +``` + +### Keywords + + CREATE, JOB + +### Best Practice \ No newline at end of file diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Load/BROKER-LOAD.md b/docs/en/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Load/BROKER-LOAD.md index 59956abc769515d..0cbfc8fd54ae9e1 100644 --- a/docs/en/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Load/BROKER-LOAD.md +++ b/docs/en/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Load/BROKER-LOAD.md @@ -175,6 +175,10 @@ WITH BROKER broker_name Whether to impose strict restrictions on data. Defaults to false. + - `partial_columns` + + Boolean type, True means that use partial column update, the default value is false, this parameter is only allowed to be set when the table model is Unique and Merge on Write is used. + - `timezone` Specify the time zone for some functions that are affected by time zones, such as `strftime/alignment_timestamp/from_unixtime`, etc. Please refer to the [timezone](../../../../advanced/time-zone.md) documentation for details. If not specified, the "Asia/Shanghai" timezone is used diff --git a/docs/en/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-PARTITION-VERSION.md b/docs/en/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-PARTITION-VERSION.md new file mode 100644 index 000000000000000..14e63036bdc3bfd --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-PARTITION-VERSION.md @@ -0,0 +1,67 @@ +--- +{ + "title": "ADMIN-SET-PARTITION-VERSION", + "language": "en" +} +--- + + + +## ADMIN-SET-PARTITION-VERSION + +### Name + +ADMIN SET PARTITION VERSION + +### Description + +This statement is used to set the version of the specified partition. + +In certain cases, the version of the partition in the metadata may not be consistent with the version of the actual replica. This command can manually set the version of the partition in the metadata. + +grammar: + +```sql +ADMIN SET TABLE table_name PARTITION VERSION + PROPERTIES ("key" = "value", ...); +``` + +The following properties are currently supported: + +1. "partition_id": Required. Specify a Partition Id. +2. "visible_version": Required. Specify Version. + +> Note: +> +> It is necessary to first confirm the version of the actual replica on the Be before set the version of the partition. This command is generally only used for emergency troubleshooting, please proceed with caution. + +### Example + +1. Set the version of partition 1769152 to 100. + +```sql +ADMIN SET TABLE tbl1 PARTITION VERSION PROPERTIES("partition_id" = "1769152", "visible_version" = "100"); +``` + +### Keywords + + ADMIN, SET, PARTITION, VERSION + +### Best Practice diff --git a/docs/en/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-TABLE-STATUS.md b/docs/en/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-TABLE-STATUS.md new file mode 100644 index 000000000000000..fde3b5ed4b66327 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-TABLE-STATUS.md @@ -0,0 +1,81 @@ +--- +{ + "title": "ADMIN-SET-TABLE-STATUS", + "language": "zh-CN" +} +--- + + + +## ADMIN-SET-TABLE-STATUS + +### Name + +ADMIN SET TABLE STATUS + +### Description + +This statement is used to set the state of the specified table. Only supports OLAP tables. + +This command is currently only used to manually set the OLAP table state to the specified state, allowing some jobs that are stuck by the table state to continue running. + +grammar: + +```sql +ADMIN SET TABLE table_name STATUS + PROPERTIES ("key" = "value", ...); +``` + +The following properties are currently supported: + +1. "state":Required. Specifying a target state then the state of the OLAP table will change to this state. + +> The current target states include: +> +> 1. NORMAL +> 2. ROLLUP +> 3. SCHEMA_CHANGE +> 4. BACKUP +> 5. RESTORE +> 6. WAITING_STABLE +> +> If the current state of the table is already the specified state, it will be ignored. + +**Note: This command is generally only used for emergency fault repair, please proceed with caution.** + +### Example + +1. Set the state of table tbl1 to NORMAL. + +```sql +admin set table tbl1 status properties("state" = "NORMAL"); +``` + +2. Set the state of table tbl2 to SCHEMA_CHANGE + +```sql +admin set table test_set_table_status status properties("state" = "SCHEMA_CHANGE"); +``` + +### Keywords + + ADMIN, SET, TABLE, STATUS + +### Best Practice \ No newline at end of file diff --git a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB-TASK.md b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB-TASK.md new file mode 100644 index 000000000000000..d8160595102e5f7 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB-TASK.md @@ -0,0 +1,74 @@ +--- +{ +"title": "SHOW-JOB-TASK", +"language": "en" +} +--- + + +## SHOW-JOB-TASK + +### Name + +SHOW JOB TASK + +### Description + +This statement is used to display the list of execution results of JOB subtasks, and the latest 20 records will be kept by default. + +grammar: + +```sql +SHOW JOB TASKS FOR job_name; +``` + + + +Result description: + +``` + JobId: JobId + TaskId: TaskId + StartTime: start execution time + EndTime: end time + Status: status + Result: execution result + ErrMsg: error message +``` + +* State + + There are the following 2 states: + * SUCCESS + * FAIL + +### Example + +1. Display the task execution list of the JOB named test1 + + ```sql + SHOW JOB TASKS FOR test1; + ``` + +###Keywords + + SHOW, JOB, TASK + +### Best Practice \ No newline at end of file diff --git a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB.md b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB.md new file mode 100644 index 000000000000000..60c1dfd1cd8ebd0 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB.md @@ -0,0 +1,94 @@ +--- +{ +"title": "SHOW-JOB", +"language": "en" +} +--- + + + +## SHOW-JOB + +###Name + +SHOW JOB + +### Description + +This statement is used to display the running status of the JOB job + +grammar: + +```sql +SHOW JOBS|JOB FOR job_name; +``` + +SHOW JOBS is used to display the running status of all jobs under the current DB, and SHOW JOB FOR job_name is used to display the running status of the specified job. + +Result description: + +``` + Id: JobId + Db: database name + Name: Job name + Definer: create user + TimeZone: time zone + ExecuteType: RECURRING means cyclic scheduling, that is, the scheduling time specified by the every statement, ONCE_TIME means a one-time task. + ExecuteAT: ONCE_TIME The execution start time of the task + ExecuteInterval: Interval of periodic scheduling tasks + ExecuteInterval: The time interval unit for periodic scheduling tasks + STARTS: The start time of periodic scheduled task settings + ENDS: The end time set by the periodic scheduling task + Status: Job status + LastExecuteFinishTime: The time when the last execution was completed + ErrorMsg: error message + Comment: Remarks + + +``` + +* State + + There are the following 5 states: + * RUNNING: running + * PAUSED: Paused + * STOPPED: end (manually triggered by the user) + * FINISHED: Finished + * WAITING_FINISH: pending completion + +### Example + +1. Display all JOBs under the current DB. + + ```sql + SHOW JOBS; + ``` + +2. Display the JOB named test1 + + ```sql + SHOW JOB FOR test1; + ``` + +###Keywords + + SHOW, JOB + +### Best Practice \ No newline at end of file diff --git a/docs/sidebars.json b/docs/sidebars.json index c1329d4f47dd7bf..89367c64acf6aa6 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -812,6 +812,8 @@ "sql-manual/sql-reference/Database-Administration-Statements/INSTALL-PLUGIN", "sql-manual/sql-reference/Database-Administration-Statements/UNINSTALL-PLUGIN", "sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-REPLICA-STATUS", + "sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-PARTITION-VERSION", + "sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-TABLE-STATUS", "sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SHOW-REPLICA-DISTRIBUTION", "sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SHOW-REPLICA-STATUS", "sql-manual/sql-reference/Database-Administration-Statements/ADMIN-REPAIR-TABLE", @@ -870,6 +872,7 @@ "sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE-AS-SELECT", "sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-EXTERNAL-TABLE", "sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-INDEX", + "sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-JOB", "sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-VIEW", "sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-MATERIALIZED-VIEW", "sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-FUNCTION", @@ -900,7 +903,10 @@ "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-WORKLOAD-GROUP", - "sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-SQL-BLOCK-RULE" + "sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-SQL-BLOCK-RULE", + "sql-manual/sql-reference/Data-Definition-Statements/Alter/PAUSE-JOB", + "sql-manual/sql-reference/Data-Definition-Statements/Alter/RESUME-JOB", + "sql-manual/sql-reference/Data-Definition-Statements/Alter/STOP-JOB" ] }, { @@ -1012,6 +1018,8 @@ "sql-manual/sql-reference/Show-Statements/SHOW-TYPECAST", "sql-manual/sql-reference/Show-Statements/SHOW-FILE", "sql-manual/sql-reference/Show-Statements/SHOW-GRANTS", + "sql-manual/sql-reference/Show-Statements/SHOW-JOB", + "sql-manual/sql-reference/Show-Statements/SHOW-JOB-TASK", "sql-manual/sql-reference/Show-Statements/SHOW-LAST-INSERT", "sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE", "sql-manual/sql-reference/Show-Statements/SHOW-LOAD-WARNINGS", diff --git a/docs/zh-CN/docs/admin-manual/http-actions/be/download.md b/docs/zh-CN/docs/admin-manual/http-actions/be/download.md index b240f48ba5be254..f27ce3f10f9a333 100644 --- a/docs/zh-CN/docs/admin-manual/http-actions/be/download.md +++ b/docs/zh-CN/docs/admin-manual/http-actions/be/download.md @@ -1,6 +1,6 @@ --- { - "title": "下载oad日志", + "title": "下载load日志", "language": "zh-CN" } --- diff --git a/docs/zh-CN/docs/admin-manual/maint-monitor/automatic-service-start.md b/docs/zh-CN/docs/admin-manual/maint-monitor/automatic-service-start.md index 63a71404112bb15..09d1fac228261d5 100644 --- a/docs/zh-CN/docs/admin-manual/maint-monitor/automatic-service-start.md +++ b/docs/zh-CN/docs/admin-manual/maint-monitor/automatic-service-start.md @@ -47,10 +47,14 @@ doris ALL=(ALL) NOPASSWD:DORISCTL ``` ### 配置步骤 +1. 分别在fe.conf和be.conf中添加 JAVA_HOME变量配置,否则使用systemctl start 将无法启动服务 + ``` + echo "JAVA_HOME=your_java_home" >> /home/doris/fe/conf/fe.conf + echo "JAVA_HOME=your_java_home" >> /home/doris/be/conf/be.conf + ``` +2. 下载doris-fe.service文件: [doris-fe.service](https://github.com/apache/doris/blob/master/tools/systemd/doris-fe.service) -1. 下载doris-fe.service文件: [doris-fe.service](https://github.com/apache/doris/blob/master/tools/systemd/doris-fe.service) - -2. doris-fe.service具体内容如下: +3. doris-fe.service具体内容如下: ``` # Licensed to the Apache Software Foundation (ASF) under one @@ -97,9 +101,9 @@ doris ALL=(ALL) NOPASSWD:DORISCTL - ExecStart、ExecStop根据实际部署的fe的路径进行配置 -3. 下载doris-be.service文件: [doris-be.service](https://github.com/apache/doris/blob/master/tools/systemd/doris-be.service) +4. 下载doris-be.service文件: [doris-be.service](https://github.com/apache/doris/blob/master/tools/systemd/doris-be.service) -4. doris-be.service具体内容如下: +5. doris-be.service具体内容如下: ``` # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -145,11 +149,11 @@ doris ALL=(ALL) NOPASSWD:DORISCTL - ExecStart、ExecStop根据实际部署的be的路径进行配置 -5. 服务配置 +6. 服务配置 将doris-fe.service、doris-be.service两个文件放到 /usr/lib/systemd/system 目录下 -6. 设置自启动 +7. 设置自启动 添加或修改配置文件后,需要重新加载 @@ -164,7 +168,7 @@ doris ALL=(ALL) NOPASSWD:DORISCTL systemctl enable doris-be ``` -7. 服务启动 +8. 服务启动 ``` systemctl start doris-fe diff --git a/docs/zh-CN/docs/advanced/cold-hot-separation.md b/docs/zh-CN/docs/advanced/cold-hot-separation.md index c79abd072d2cb06..51693c3ec32ae46 100644 --- a/docs/zh-CN/docs/advanced/cold-hot-separation.md +++ b/docs/zh-CN/docs/advanced/cold-hot-separation.md @@ -101,6 +101,7 @@ ALTER TABLE create_table_not_have_policy set ("storage_policy" = "test_policy"); ``` ALTER TABLE create_table_partition MODIFY PARTITION (*) SET("storage_policy"="test_policy"); ``` +**注意**,如果用户在建表时给整张table和部分partition指定了不同的storage policy,partition设置的storage policy会被无视,整张表的所有partition都会使用table的policy. 如果您需要让某个partition的policy和别的不同,则可以使用上文中对一个已存在的partition,关联storage policy的方式修改. 具体可以参考docs目录下[resource](../sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-RESOURCE.md)、 [policy](../sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-POLICY.md)、 [create table](../sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md)、 [alter table](../sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-COLUMN.md)等文档,里面有详细介绍 ### 一些限制 diff --git a/docs/zh-CN/docs/advanced/variables.md b/docs/zh-CN/docs/advanced/variables.md index c07185dc1f74952..4587f86e81629b9 100644 --- a/docs/zh-CN/docs/advanced/variables.md +++ b/docs/zh-CN/docs/advanced/variables.md @@ -385,7 +385,7 @@ SELECT /*+ SET_VAR(query_timeout = 1, enable_partition_cache=true) */ sleep(3); - `sql_select_limit` - 用于兼容 MySQL 客户端。无实际作用。 + 用于设置 select 语句的默认返回行数,包括 insert 语句的 select 从句。默认不限制。 - `system_time_zone` diff --git a/docs/zh-CN/docs/data-operate/import/import-way/stream-load-manual.md b/docs/zh-CN/docs/data-operate/import/import-way/stream-load-manual.md index 115cc954b8bea4f..36913224760ca52 100644 --- a/docs/zh-CN/docs/data-operate/import/import-way/stream-load-manual.md +++ b/docs/zh-CN/docs/data-operate/import/import-way/stream-load-manual.md @@ -253,16 +253,16 @@ Stream Load 由于使用的是 HTTP 协议,所以所有导入任务有关的 ### 使用SQL表达Stream Load的参数 -可以在Header中添加一个`sql`的参数,去替代之前参数中的`column_separator`、`line_delimiter`、`where`、`columns`参数,方便使用。 +可以在Header中添加一个`sql`的参数,去替代之前参数中的`column_separator`、`line_delimiter`、`where`、`columns`等参数,方便使用。 ``` -curl --location-trusted -u user:passwd [-H "sql: ${load_sql}"...] -T data.file -XPUT http://fe_host:http_port/api/{db}/{table}/_stream_load_with_sql +curl --location-trusted -u user:passwd [-H "sql: ${load_sql}"...] -T data.file -XPUT http://fe_host:http_port/api/_http_stream # -- load_sql -# insert into db.table (col, ...) select stream_col, ... from stream("property1"="value1"); +# insert into db.table (col, ...) select stream_col, ... from http_stream("property1"="value1"); -# stream +# http_stream # ( # "column_separator" = ",", # "format" = "CSV", @@ -273,7 +273,7 @@ curl --location-trusted -u user:passwd [-H "sql: ${load_sql}"...] -T data.file - 示例: ``` -curl --location-trusted -u root: -T test.csv -H "sql:insert into demo.example_tbl_1(user_id, age, cost) select c1, c4, c7 * 2 from stream("format" = "CSV", "column_separator" = "," ) where age >= 30" http://127.0.0.1:28030/api/demo/example_tbl_1/_stream_load_with_sql +curl --location-trusted -u root: -T test.csv -H "sql:insert into demo.example_tbl_1(user_id, age, cost) select c1, c4, c7 * 2 from http_stream("format" = "CSV", "column_separator" = "," ) where age >= 30" http://127.0.0.1:28030/api/_http_stream ``` diff --git a/docs/zh-CN/docs/data-table/index/inverted-index.md b/docs/zh-CN/docs/data-table/index/inverted-index.md index 25633f091366d4f..01ee5d1ea923b0d 100644 --- a/docs/zh-CN/docs/data-table/index/inverted-index.md +++ b/docs/zh-CN/docs/data-table/index/inverted-index.md @@ -167,6 +167,51 @@ SELECT * FROM table_name WHERE ts > '2023-01-01 00:00:00'; SELECT * FROM table_name WHERE op_type IN ('add', 'delete'); ``` +- 分词函数 + +如果想检查分词实际效果或者对一段文本进行分词的话,可以使用tokenize函数 +```sql +mysql> SELECT TOKENIZE('武汉长江大桥','"parser"="chinese","parser_mode"="fine_grained"'); ++-----------------------------------------------------------------------------------+ +| tokenize('武汉长江大桥', '"parser"="chinese","parser_mode"="fine_grained"') | ++-----------------------------------------------------------------------------------+ +| ["武汉", "武汉长江大桥", "长江", "长江大桥", "大桥"] | ++-----------------------------------------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> SELECT TOKENIZE('武汉市长江大桥','"parser"="chinese","parser_mode"="fine_grained"'); ++--------------------------------------------------------------------------------------+ +| tokenize('武汉市长江大桥', '"parser"="chinese","parser_mode"="fine_grained"') | ++--------------------------------------------------------------------------------------+ +| ["武汉", "武汉市", "市长", "长江", "长江大桥", "大桥"] | ++--------------------------------------------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> SELECT TOKENIZE('武汉市长江大桥','"parser"="chinese","parser_mode"="coarse_grained"'); ++----------------------------------------------------------------------------------------+ +| tokenize('武汉市长江大桥', '"parser"="chinese","parser_mode"="coarse_grained"') | ++----------------------------------------------------------------------------------------+ +| ["武汉市", "长江大桥"] | ++----------------------------------------------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> SELECT TOKENIZE('I love CHINA','"parser"="english"'); ++------------------------------------------------+ +| tokenize('I love CHINA', '"parser"="english"') | ++------------------------------------------------+ +| ["i", "love", "china"] | ++------------------------------------------------+ +1 row in set (0.02 sec) + +mysql> SELECT TOKENIZE('I love CHINA 我爱我的祖国','"parser"="unicode"'); ++-------------------------------------------------------------------+ +| tokenize('I love CHINA 我爱我的祖国', '"parser"="unicode"') | ++-------------------------------------------------------------------+ +| ["i", "love", "china", "我", "爱", "我", "的", "祖", "国"] | ++-------------------------------------------------------------------+ +1 row in set (0.02 sec) +``` + ## 使用示例 用hackernews 100万条数据展示倒排索引的创建、全文检索、普通查询,包括跟无索引的查询性能进行简单对比。 diff --git a/docs/zh-CN/docs/ecosystem/flink-doris-connector.md b/docs/zh-CN/docs/ecosystem/flink-doris-connector.md index 179234b60613649..a170f3b02e76e69 100644 --- a/docs/zh-CN/docs/ecosystem/flink-doris-connector.md +++ b/docs/zh-CN/docs/ecosystem/flink-doris-connector.md @@ -91,7 +91,7 @@ CREATE TABLE flink_doris_source ( ) WITH ( 'connector' = 'doris', - 'fenodes' = 'FE_IP:8030', + 'fenodes' = 'FE_IP:HTTP_PORT', 'table.identifier' = 'database.table', 'username' = 'root', 'password' = 'password' @@ -102,7 +102,7 @@ CREATE TABLE flink_doris_source ( ```java DorisOptions.Builder builder = DorisOptions.builder() - .setFenodes("FE_IP:8030") + .setFenodes("FE_IP:HTTP_PORT") .setTableIdentifier("db.table") .setUsername("root") .setPassword("password"); @@ -133,7 +133,7 @@ CREATE TABLE flink_doris_sink ( ) WITH ( 'connector' = 'doris', - 'fenodes' = 'FE_IP:8030', + 'fenodes' = 'FE_IP:HTTP_PORT', 'table.identifier' = 'db.table', 'username' = 'root', 'password' = 'password', @@ -158,7 +158,7 @@ env.setRuntimeMode(RuntimeExecutionMode.BATCH); DorisSink.Builder builder = DorisSink.builder(); DorisOptions.Builder dorisBuilder = DorisOptions.builder(); -dorisBuilder.setFenodes("FE_IP:8030") +dorisBuilder.setFenodes("FE_IP:HTTP_PORT") .setTableIdentifier("db.table") .setUsername("root") .setPassword("password"); @@ -192,7 +192,7 @@ env.setRuntimeMode(RuntimeExecutionMode.BATCH); //doris sink option DorisSink.Builder builder = DorisSink.builder(); DorisOptions.Builder dorisBuilder = DorisOptions.builder(); -dorisBuilder.setFenodes("FE_IP:8030") +dorisBuilder.setFenodes("FE_IP:HTTP_PORT") .setTableIdentifier("db.table") .setUsername("root") .setPassword("password"); @@ -303,15 +303,16 @@ ON a.city = c.city ### 通用配置项 -| Key | Default Value | Required | Comment | -| -------------------------------- | ------------- | -------- | ----------------------------------------------- | -| fenodes | -- | Y | Doris FE http 地址, 支持多个地址,使用逗号分隔 | -| table.identifier | -- | Y | Doris 表名,如:db.tbl | -| username | -- | Y | 访问 Doris 的用户名 | -| password | -- | Y | 访问 Doris 的密码 | -| doris.request.retries | 3 | N | 向 Doris 发送请求的重试次数 | -| doris.request.connect.timeout.ms | 30000 | N | 向 Doris 发送请求的连接超时时间 | -| doris.request.read.timeout.ms | 30000 | N | 向 Doris 发送请求的读取超时时间 | +| Key | Default Value | Required | Comment | +|----------------------------------|---------------|----------|----------------------------------------------------------------------------------------------------| +| fenodes | -- | Y | Doris FE http 地址, 支持多个地址,使用逗号分隔 | +| benodes | -- | N | Doris BE http 地址, 支持多个地址,使用逗号分隔,参考[#187](https://github.com/apache/doris-flink-connector/pull/187) | +| table.identifier | -- | Y | Doris 表名,如:db.tbl | +| username | -- | Y | 访问 Doris 的用户名 | +| password | -- | Y | 访问 Doris 的密码 | +| doris.request.retries | 3 | N | 向 Doris 发送请求的重试次数 | +| doris.request.connect.timeout.ms | 30000 | N | 向 Doris 发送请求的连接超时时间 | +| doris.request.read.timeout.ms | 30000 | N | 向 Doris 发送请求的读取超时时间 | ### Source 配置项 @@ -441,6 +442,8 @@ insert into doris_sink select id,name from cdc_mysql_source; - **--oracle-conf** Oracle CDCSource 配置,例如--oracle-conf hostname=127.0.0.1 ,您可以在[这里](https://ververica.github.io/flink-cdc-connectors/master/content/connectors/oracle-cdc.html)查看所有配置Oracle-CDC,其中hostname/username/password/database-name/schema-name 是必需的。 - **--sink-conf** Doris Sink 的所有配置,可以在[这里](https://doris.apache.org/zh-CN/docs/dev/ecosystem/flink-doris-connector/#%E9%80%9A%E7%94%A8%E9%85%8D%E7%BD%AE%E9%A1%B9)查看完整的配置项。 - **--table-conf** Doris表的配置项,即properties中包含的内容。 例如 --table-conf replication_num=1 +- **--ignore-default-value** 关闭同步mysql表结构的默认值。适用于同步mysql数据到doris时,字段有默认值,但实际插入数据为null情况。参考[#152](https://github.com/apache/doris-flink-connector/pull/152) +- **--use-new-schema-change** 新的schema change支持同步mysql多列变更、默认值。参考[#167](https://github.com/apache/doris-flink-connector/pull/167) >注:同步时需要在$FLINK_HOME/lib 目录下添加对应的Flink CDC依赖,比如 flink-sql-connector-mysql-cdc-${version}.jar,flink-sql-connector-oracle-cdc-${version}.jar diff --git a/docs/zh-CN/docs/ecosystem/udf/java-user-defined-function.md b/docs/zh-CN/docs/ecosystem/udf/java-user-defined-function.md index a30fcca614d8cac..e7e37b56e5a0898 100644 --- a/docs/zh-CN/docs/ecosystem/udf/java-user-defined-function.md +++ b/docs/zh-CN/docs/ecosystem/udf/java-user-defined-function.md @@ -57,8 +57,9 @@ Java UDF 为用户提供UDF编写的Java接口,以方便用户使用Java语言 |String|String| |Decimal|BigDecimal| |```array```|```ArrayList```| +|```map```|```HashMap```| -* array类型可以嵌套基本类型,例如Doris: ```array```对应JAVA UDF Argument Type: ```ArrayList```, 其他依此类推 +* array/map类型可以嵌套基本类型,例如Doris: ```array```对应JAVA UDF Argument Type: ```ArrayList```, 其他依此类推 ## 编写 UDF 函数 本小节主要介绍如何开发一个 Java UDF。在 `samples/doris-demo/java-udf-demo/` 下提供了示例,可供参考,查看点击[这里](https://github.com/apache/doris/tree/master/samples/doris-demo/java-udf-demo) diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/date_ceil.md b/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/date_ceil.md new file mode 100644 index 000000000000000..eefe578262c9055 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/date_ceil.md @@ -0,0 +1,96 @@ +--- +{ + "title": "date_ceil", + "language": "zh-CN" +} +--- + + + +## date_ceil +### description +#### Syntax + +`DATETIME DATE_CEIL(DATETIME datetime, INTERVAL period type)` + + +将日期转化为指定的时间间隔周期的最近上取整时刻。 + +datetime 参数是合法的日期表达式。 + +period 参数是指定每个周期有多少个单位组成,开始的时间起点为0001-01-01T00:00:00. + +type 参数可以是下列值:YEAR, MONTH, DAY, HOUR, MINUTE, SECOND. + +### example + +``` +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 second); ++--------------------------------------------------------------+ +| second_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++--------------------------------------------------------------+ +| 2023-07-13 22:28:20 | ++--------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 minute); ++--------------------------------------------------------------+ +| minute_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++--------------------------------------------------------------+ +| 2023-07-13 22:30:00 | ++--------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 hour); ++------------------------------------------------------------+ +| hour_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++------------------------------------------------------------+ +| 2023-07-13 23:00:00 | ++------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 day); ++-----------------------------------------------------------+ +| day_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++-----------------------------------------------------------+ +| 2023-07-15 00:00:00 | ++-----------------------------------------------------------+ +1 row in set (0.00 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 month); ++-------------------------------------------------------------+ +| month_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++-------------------------------------------------------------+ +| 2023-12-01 00:00:00 | ++-------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql [(none)]>select date_ceil("2023-07-13 22:28:18",interval 5 year); ++------------------------------------------------------------+ +| year_ceil('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++------------------------------------------------------------+ +| 2026-01-01 00:00:00 | ++------------------------------------------------------------+ +1 row in set (0.00 sec) +``` + +### keywords + + DATE_CEIL,DATE,CEIL diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/date_floor.md b/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/date_floor.md new file mode 100644 index 000000000000000..53ef6003fd0994e --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/date-time-functions/date_floor.md @@ -0,0 +1,104 @@ +--- +{ + "title": "date_floor", + "language": "zh-CN" +} +--- + + + +## date_floor +### description +#### Syntax + +`DATETIME DATE_FLOOR(DATETIME datetime, INTERVAL period type)` + + +将日期转化为指定的时间间隔周期的最近下取整时刻。 + +datetime 参数是合法的日期表达式。 + +period 参数是指定每个周期有多少个单位组成,开始的时间起点为0001-01-01T00:00:00. + +type 参数可以是下列值:YEAR, MONTH, DAY, HOUR, MINUTE, SECOND. + +### example + +``` +mysql>select date_floor("0001-01-01 00:00:16",interval 5 second); ++---------------------------------------------------------------+ +| second_floor('0001-01-01 00:00:16', 5, '0001-01-01 00:00:00') | ++---------------------------------------------------------------+ +| 0001-01-01 00:00:15 | ++---------------------------------------------------------------+ +1 row in set (0.00 sec) + +mysql>select date_floor("0001-01-01 00:00:18",interval 5 second); ++---------------------------------------------------------------+ +| second_floor('0001-01-01 00:00:18', 5, '0001-01-01 00:00:00') | ++---------------------------------------------------------------+ +| 0001-01-01 00:00:15 | ++---------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 minute); ++---------------------------------------------------------------+ +| minute_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++---------------------------------------------------------------+ +| 2023-07-13 22:25:00 | ++---------------------------------------------------------------+ +1 row in set (0.00 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 hour); ++-------------------------------------------------------------+ +| hour_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++-------------------------------------------------------------+ +| 2023-07-13 18:00:00 | ++-------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 day); ++------------------------------------------------------------+ +| day_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++------------------------------------------------------------+ +| 2023-07-10 00:00:00 | ++------------------------------------------------------------+ +1 row in set (0.00 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 month); ++--------------------------------------------------------------+ +| month_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++--------------------------------------------------------------+ +| 2023-07-01 00:00:00 | ++--------------------------------------------------------------+ +1 row in set (0.01 sec) + +mysql>select date_floor("2023-07-13 22:28:18",interval 5 year); ++-------------------------------------------------------------+ +| year_floor('2023-07-13 22:28:18', 5, '0001-01-01 00:00:00') | ++-------------------------------------------------------------+ +| 2021-01-01 00:00:00 | ++-------------------------------------------------------------+ + +``` + +### keywords + + DATE_FLOOR,DATE,FLOOR diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/hdfs.md b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/hdfs.md index 47e723a623fe49e..c7faaa7a86f46f4 100644 --- a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/hdfs.md +++ b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/hdfs.md @@ -70,7 +70,7 @@ hdfs( - `dfs.client.failover.proxy.provider.your-nameservices`:(选填) 文件格式相关参数 -- `format`:(必填) 目前支持 `csv/csv_with_names/csv_with_names_and_types/json/parquet/orc` +- `format`:(必填) 目前支持 `csv/csv_with_names/csv_with_names_and_types/json/parquet/orc/avro` - `column_separator`:(选填) 列分割符, 默认为`,`。 - `line_delimiter`:(选填) 行分割符,默认为`\n`。 - `compress_type`: (选填) 目前支持 `UNKNOWN/PLAIN/GZ/LZO/BZ2/LZ4FRAME/DEFLATE`。 默认值为 `UNKNOWN`, 将会根据 `uri` 的后缀自动推断类型。 diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/s3.md b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/s3.md index 1a642056432c9c2..081734985ca4903 100644 --- a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/s3.md +++ b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/s3.md @@ -428,6 +428,24 @@ MySQL [(none)]> select * from s3( | 5 | forest brown coral puff cream | Manufacturer#3 | Brand#32 | STANDARD POLISHED TIN | 15 | SM PKG | 905 | wake carefully | +-----------+------------------------------------------+----------------+----------+-------------------------+--------+-------------+---------------+---------------------+ ``` +**avro format** + +`avro` 格式:S3 tvf支持从avro文件中解析出table schema的列名、列类型。举例: + +```sql +select * from s3( + "uri" = "http://127.0.0.1:9312/test2/person.avro", + "ACCESS_KEY" = "ak", + "SECRET_KEY" = "sk", + "FORMAT" = "avro"); ++--------+--------------+-------------+-----------------+ +| name | boolean_type | double_type | long_type | ++--------+--------------+-------------+-----------------+ +| Alyssa | 1 | 10.0012 | 100000000221133 | +| Ben | 0 | 5555.999 | 4009990000 | +| lisi | 0 | 5992225.999 | 9099933330 | ++--------+--------------+-------------+-----------------+ +``` **uri包含通配符** diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md index 43987d00e54df58..c9ea4a4c7d32bf6 100644 --- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md @@ -64,8 +64,8 @@ partition_desc ["key"="value"] - 如指定分桶方式,只能修改分桶数,不可修改分桶方式或分桶列。如果指定了分桶方式,但是没有指定分桶数,则分桶数会使用默认值10,不会使用建表时指定的分桶数。如果要指定分桶数,则必须指定分桶方式。 - ["key"="value"] 部分可以设置分区的一些属性,具体说明见 [CREATE TABLE](../Create/CREATE-TABLE.md) - 如果建表时用户未显式创建Partition,则不支持通过ALTER的方式增加分区 -- 如果用户使用的是List Partition则可以增加default partition - - ALTER TABLE ADD PARTITION DEFAULT +- 如果用户使用的是List Partition则可以增加default partition,default partition将会存储所有不满足其他分区键要求的数据。 + - ALTER TABLE table_name ADD PARTITION partition_name 2. 删除分区 diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/PAUSE-JOB.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/PAUSE-JOB.md new file mode 100644 index 000000000000000..cfc88c8f6dea79f --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/PAUSE-JOB.md @@ -0,0 +1,54 @@ +--- +{ +"title": "PAUSE-JOB", +"language": "zh-CN" +} +--- + + + +## PAUSE-JOB + +### Name + +PAUSE JOB + +### Description + +用户暂停一个 JOB 作业。被停止的作业可以通过 RESUME JOB 恢复。 + +```sql +PAUSE JOB FOR job_name; +``` + +### Example + +1. 暂停名称为 test1 的作业。 + +```sql + PAUSE JOB FOR test1; +``` + +### Keywords + + PAUSE, JOB + +### Best Practice + diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/RESUME-JOB.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/RESUME-JOB.md new file mode 100644 index 000000000000000..2b35f8685e58cf3 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/RESUME-JOB.md @@ -0,0 +1,54 @@ +--- +{ +"title": "RESUME-JOB", +"language": "zh-CN" +} +--- + + + +## RESUME-JOB + +### Name + +RESUME JOB + +### Description + +用于重启一个 PAUSE 状态的 JOB 作业。重启的作业,将继续按照周期执行。STOP 状态的 JOB 无法被恢复。 + +```sql +RESUME JOB FOR job_name; +``` + +### Example + +1. 重启名称为 test1 的 JOB。 + + ```sql + RESUME JOB FOR test1; + ``` + +### Keywords + + RESUME, JOB + +### Best Practice + diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/STOP-JOB.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/STOP-JOB.md new file mode 100644 index 000000000000000..8a6a05b86fef94b --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/STOP-JOB.md @@ -0,0 +1,54 @@ +--- +{ +"title": "STOP-JOB", +"language": "zh-CN" +} +--- + + + +## STOP-JOB + +### Name + +STOP JOB + +### Description + +用户停止一个 JOB 作业。被停止的作业无法再重新运行。 + +```sql +STOP JOB FOR job_name; +``` + +### Example + +1. 停止名称为 test1 的作业。 + + ```sql + STOP JOB FOR test1; + ``` + +### Keywords + + STOP, JOB + +### Best Practice + diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-JOB.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-JOB.md new file mode 100644 index 000000000000000..c06d48a4d0110ae --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-JOB.md @@ -0,0 +1,130 @@ +--- +{ +"title": "CREATE-JOB", +"language": "zh-CN" +} + +--- + + + +## CREATE-JOB + +### Name + +CREATE JOB + +### Description + +Doris Job 是根据既定计划运行的任务,用于在特定时间或指定时间间隔触发预定义的操作,从而帮助我们自动执行一些任务。从功能上来讲,它类似于操作系统上的 +定时任务(如:Linux 中的 cron、Windows 中的计划任务)。但 Doris 的 Job 调度可以精确到秒级。 + +Job 有两种类型:`ONE_TIME` 和 `BATCH`。其中 `ONE_TIME` 类型的 Job 会在指定的时间点触发,它主要用于一次性任务,而 `BATCH` 类型的 Job 会在指定的时间间隔内循环触发。 +主要用于周期性执行的任务。 + +目前仅支持 ***ADMIN*** 权限执行此操作。 + + 语法: + +```sql +CREATE + JOB + job_name + ON SCHEDULE schedule + [COMMENT 'string'] + DO sql_body; + +schedule: { + AT timestamp + | EVERY interval + [STARTS timestamp ] + [ENDS timestamp ] +} + +interval: + quantity { DAY | HOUR | MINUTE | + WEEK | SECOND } +``` + +一条有效的 Job 语句必须包含以下内容 + +- 关键字 CREATE JOB 加上作业名称,它在一个 db 中标识唯一事件。 +- ON SCHEDULE 子句,它指定了 Job 作业的类型和触发时间以及频率。 +- DO 子句,它指定了 Job 作业触发时需要执行的操作。 + +这是一个最简单的例子: + +```sql +CREATE JOB my_job ON SCHEDULE EVERY 1 MINUTE DO INSERT INTO db1.tbl1 SELECT * FROM db2.tbl2; +``` + +该语句表示创建一个名为 my_job 的作业,每分钟执行一次,执行的操作是将 db2.tbl2 中的数据导入到 db1.tbl1 中。 + +SCHEDULER 语句用于定义作业的执行时间,频率以及持续时间,它可以指定一次性作业或者周期性作业。 +- AT timestamp + + 用于一次性事件,它指定事件仅在 给定的日期和时间执行一次 timestamp,该日期和时间必须包含日期和时间 + +- EVERY + + 表示定期重复操作,它指定了作业的执行频率,关键字后面要指定一个时间间隔,该时间间隔可以是天、小时、分钟、秒、周。 + + - interval + + 用于指定作业执行频率,它可以是天、小时、分钟、秒、周。例如:` 1 DAY` 表示每天执行一次,` 1 HOUR` 表示每小时执行一次,` 1 MINUTE` 表示每分钟执行一次,` 1 WEEK` 表示每周执行一次,` 1 SECOND` 表示每秒执行一次。 + + - STARTS timestamp + + 用于指定作业的开始时间,如果没有指定,则从当前时间的下一个时间点开始执行。 + + - ENDS timestamp + + 用于指定作业的结束时间,如果没有指定,则表示永久执行。 +- DO + + 用于指定作业触发时需要执行的操作,目前支持所有的 ***INSERT,UPDATE*** 操作。后续我们会支持更多的操作。 + +### Example + +创建一个一次性的 Job,它会在 2020-01-01 00:00:00 时执行一次,执行的操作是将 db2.tbl2 中的数据导入到 db1.tbl1 中。 + +```sql + +CREATE JOB my_job ON SCHEDULE AT '2020-01-01 00:00:00' DO INSERT INTO db1.tbl1 SELECT * FROM db2.tbl2; + +``` + +创建一个周期性的 Job,它会在 2020-01-01 00:00:00 时开始执行,每天执行一次,执行的操作是将 db2.tbl2 中的数据导入到 db1.tbl1 中。 + +```sql +CREATE JOB my_job ON SCHEDULE EVERY 1 DAY STARTS '2020-01-01 00:00:00' DO INSERT INTO db1.tbl1 SELECT * FROM db2.tbl2 WHERE create_time >= days_add(now(),-1); +``` + +创建一个周期性的 Job,它会在 2020-01-01 00:00:00 时开始执行,每天执行一次,执行的操作是将 db2.tbl2 中的数据导入到 db1.tbl1 中,该 Job 在 2020-01-01 00:10:00 时结束。 + +```sql +CREATE JOB my_job ON SCHEDULER EVERY 1 DAY STARTS '2020-01-01 00:00:00' ENDS '2020-01-01 00:10:00' DO INSERT INTO db1.tbl1 SELECT * FROM db2.tbl2 create_time >= days_add(now(),-1); +``` + +### Keywords + + CREATE, JOB + +### Best Practice \ No newline at end of file diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Load/BROKER-LOAD.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Load/BROKER-LOAD.md index 36dfa68c23ea7fb..eef30f32a167237 100644 --- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Load/BROKER-LOAD.md +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Load/BROKER-LOAD.md @@ -175,6 +175,10 @@ WITH BROKER broker_name 是否对数据进行严格限制。默认为 false。 + - `partial_columns` + + 布尔类型,为 true 表示使用部分列更新,默认值为 false,该参数只允许在表模型为 Unique 且采用 Merge on Write 时设置。 + - `timezone` 指定某些受时区影响的函数的时区,如 `strftime/alignment_timestamp/from_unixtime` 等等,具体请查阅 [时区](../../../../advanced/time-zone.md) 文档。如果不指定,则使用 "Asia/Shanghai" 时区 diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-PARTITION-VERSION.md b/docs/zh-CN/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-PARTITION-VERSION.md new file mode 100644 index 000000000000000..3ce8d43cd685a98 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-PARTITION-VERSION.md @@ -0,0 +1,67 @@ +--- +{ + "title": "ADMIN-SET-PARTITION-VERSION", + "language": "zh-CN" +} +--- + + + +## ADMIN-SET-PARTITION-VERSION + +### Name + +ADMIN SET PARTITION VERSION + +### Description + +该语句用于手动改变指定分区的可见版本。 + +在某些特殊情况下,元数据中分区的版本有可能和实际副本的版本不一致,该命令可手动改变元数据中分区的版本。 + +语法: + +```sql +ADMIN SET TABLE table_name PARTITION VERSION + PROPERTIES ("key" = "value", ...); +``` + +目前支持如下属性: + +1. "partition_id":必需。指定一个 Partition Id. +2. "visible_version":必需。指定 Version. + +> 注意: +> +> 设置分区的版本需要先确认Be机器上实际副本的版本,此命令一般只用于紧急故障修复,请谨慎操作。 + +### Example + +1. 设置 partition 1769152 在 FE 元数据上的版本为 100。 + +```sql +ADMIN SET TABLE tbl1 PARTITION VERSION PROPERTIES("partition_id" = "1769152", "visible_version" = "100"); +``` + +### Keywords + + ADMIN, SET, PARTITION, VERSION + +### Best Practice diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-TABLE-STATUS.md b/docs/zh-CN/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-TABLE-STATUS.md new file mode 100644 index 000000000000000..8a0d69503d87723 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Database-Administration-Statements/ADMIN-SET-TABLE-STATUS.md @@ -0,0 +1,84 @@ +--- +{ + "title": "ADMIN-SET-TABLE-STATUS", + "language": "zh-CN" +} +--- + + + +## ADMIN-SET-TABLE-STATUS + +### Name + +ADMIN SET TABLE STATUS + +### Description + +该语句用于设置指定表的状态,仅支持OLAP表。 + +该命令目前仅用于手动将 OLAP 表状态设置为指定状态,从而使得某些由于表状态被阻碍的任务能够继续运行。 + +语法: + +```sql +ADMIN SET TABLE table_name STATUS + PROPERTIES ("key" = "value", ...); +``` + +目前支持以下属性: + +1. "state":必需。指定一个目标状态,将会修改 OLAP 表的状态至此状态。 + +> 当前可修改的目标状态包括: +> +> 1. NORMAL +> 2. ROLLUP +> 3. SCHEMA_CHANGE +> 4. BACKUP +> 5. RESTORE +> 6. WAITING_STABLE +> +> 如果表的状态已经是指定的状态,则会被忽略。 + +**注意:此命令一般只用于紧急故障修复,请谨慎操作。** + +### Example + +1. 设置表 tbl1 的状态为 NORMAL。 + +```sql +admin set table tbl1 status properties("state" = "NORMAL"); +``` + +2. 设置表 tbl2 的状态为 SCHEMA_CHANGE。 + +```sql +admin set table test_set_table_status status properties("state" = "SCHEMA_CHANGE"); +``` + +### Keywords + + ADMIN, SET, TABLE, STATUS + +### Best Practice + + + diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB-TASK.md b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB-TASK.md new file mode 100644 index 000000000000000..d1b458ea29c9dd4 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB-TASK.md @@ -0,0 +1,75 @@ +--- +{ +"title": "SHOW-JOB-TASK", +"language": "zh-CN" +} +--- + + + +## SHOW-JOB-TASK + +### Name + +SHOW JOB TASK + +### Description + +该语句用于展示 JOB 子任务的执行结果列表, 默认会保留最新的 20 条记录。 + +语法: + +```sql +SHOW JOB TASKS FOR job_name; +``` + + + +结果说明: + +``` + JobId: JobId + TaskId: TaskId + StartTime: 开始执行时间 + EndTime: 结束时间 + Status: 状态 + Result: 执行结果 + ErrMsg: 错误信息 +``` + +* State + + 有以下 2 种 State: + * SUCCESS + * FAIL + +### Example + +1. 展示名称为 test1 的 JOB 的任务执行列表 + + ```sql + SHOW JOB TASKS FOR test1; + ``` + +### Keywords + + SHOW, JOB, TASK + +### Best Practice diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB.md b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB.md new file mode 100644 index 000000000000000..009e2cb4e4191f0 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-JOB.md @@ -0,0 +1,94 @@ +--- +{ +"title": "SHOW-JOB", +"language": "zh-CN" +} +--- + + + +## SHOW-JOB + +### Name + +SHOW JOB + +### Description + +该语句用于展示 JOB 作业运行状态 + +语法: + +```sql +SHOW JOBS|JOB FOR job_name; +``` + +SHOW JOBS 用于展示当前 DB 下所有作业的运行状态,SHOW JOB FOR job_name 用于展示指定作业的运行状态。 + +结果说明: + +``` + Id: JobId + Db: 数据库名称 + Name: Job名称 + Definer: 创建用户 + TimeZone: 时区 + ExecuteType: RECURRING 表示循环调度,即使用 every 语句指定的调度时间,ONCE_TIME 表示一次性任务。 + ExecuteAT: ONCE_TIME 任务的执行开始时间 + ExecuteInterval: 周期调度任务的间隔 + ExecuteInterval: 周期调度任务的时间间隔单位 + STARTS: 周期调度任务设置的开始时间 + ENDS: 周期调度任务设置的结束时间 + Status: Job 状态 + LastExecuteFinishTime: 上一次执行完成时间 + ErrorMsg: 错误信息 + Comment: 备注 + + +``` + +* State + + 有以下5种State: + * RUNNING:运行中 + * PAUSED:暂停 + * STOPPED:结束(用户手动触发) + * FINISHED: 完成 + * WAITING_FINISH: 待结束 + +### Example + +1. 展示当前 DB 下的所有 JOB。 + + ```sql + SHOW JOBS; + ``` + +2. 展示名称为 test1 的 JOB + + ```sql + SHOW JOB FOR test1; + ``` + +### Keywords + + SHOW, JOB + +### Best Practice diff --git a/env.sh b/env.sh index 1a832d3a5b9876d..835d8cafc92a23f 100755 --- a/env.sh +++ b/env.sh @@ -144,6 +144,25 @@ elif [[ "${DORIS_TOOLCHAIN}" == "clang" ]]; then if [[ -f "${DORIS_CLANG_HOME}/bin/llvm-symbolizer" ]]; then export ASAN_SYMBOLIZER_PATH="${DORIS_CLANG_HOME}/bin/llvm-symbolizer" fi + + covs=() + while IFS='' read -r line; do covs+=("${line}"); done <<<"$(find "${DORIS_CLANG_HOME}" -name "llvm-cov*")" + if [[ ${#covs[@]} -ge 1 ]]; then + LLVM_COV="${covs[0]}" + else + LLVM_COV="$(command -v llvm-cov)" + fi + export LLVM_COV + + profdatas=() + while IFS='' read -r line; do profdatas+=("${line}"); done <<<"$(find "${DORIS_CLANG_HOME}" -name "llvm-profdata*")" + if [[ ${#profdatas[@]} -ge 1 ]]; then + LLVM_PROFDATA="${profdatas[0]}" + else + LLVM_PROFDATA="$(command -v llvm-profdata)" + fi + export LLVM_PROFDATA + if [[ -z "${ENABLE_PCH}" ]]; then ENABLE_PCH='ON' fi diff --git a/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/utils/UdfUtils.java b/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/utils/UdfUtils.java index 5f7c05013663842..1eb0c8d07d1451b 100644 --- a/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/utils/UdfUtils.java +++ b/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/utils/UdfUtils.java @@ -19,17 +19,12 @@ import org.apache.doris.catalog.ArrayType; import org.apache.doris.catalog.MapType; -import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Type; import org.apache.doris.common.Pair; import org.apache.doris.common.exception.InternalException; import org.apache.doris.thrift.TPrimitiveType; -import org.apache.doris.thrift.TScalarType; -import org.apache.doris.thrift.TTypeDesc; -import org.apache.doris.thrift.TTypeNode; -import com.google.common.base.Preconditions; import com.google.common.collect.Sets; import com.vesoft.nebula.client.graph.data.DateTimeWrapper; import com.vesoft.nebula.client.graph.data.DateWrapper; @@ -98,6 +93,7 @@ public enum JavaUdfDataType { DECIMAL128("DECIMAL128", TPrimitiveType.DECIMAL128I, 16), ARRAY_TYPE("ARRAY_TYPE", TPrimitiveType.ARRAY, 0), MAP_TYPE("MAP_TYPE", TPrimitiveType.MAP, 0); + private final String description; private final TPrimitiveType thriftType; private final int len; @@ -106,6 +102,9 @@ public enum JavaUdfDataType { private Type itemType; private Type keyType; private Type valueType; + private int keyScale; + private int valueScale; + JavaUdfDataType(String description, TPrimitiveType thriftType, int len) { this.description = description; this.thriftType = thriftType; @@ -212,63 +211,22 @@ public void setKeyType(Type type) { public void setValueType(Type type) { this.valueType = type; } - } - public static Pair fromThrift(TTypeDesc typeDesc, int nodeIdx) throws InternalException { - TTypeNode node = typeDesc.getTypes().get(nodeIdx); - Type type = null; - switch (node.getType()) { - case SCALAR: { - Preconditions.checkState(node.isSetScalarType()); - TScalarType scalarType = node.getScalarType(); - if (scalarType.getType() == TPrimitiveType.CHAR) { - Preconditions.checkState(scalarType.isSetLen()); - type = ScalarType.createCharType(scalarType.getLen()); - } else if (scalarType.getType() == TPrimitiveType.VARCHAR) { - Preconditions.checkState(scalarType.isSetLen()); - type = ScalarType.createVarcharType(scalarType.getLen()); - } else if (scalarType.getType() == TPrimitiveType.DECIMALV2) { - Preconditions.checkState(scalarType.isSetPrecision() - && scalarType.isSetScale()); - type = ScalarType.createDecimalType(scalarType.getPrecision(), - scalarType.getScale()); - } else if (scalarType.getType() == TPrimitiveType.DECIMAL32 - || scalarType.getType() == TPrimitiveType.DECIMAL64 - || scalarType.getType() == TPrimitiveType.DECIMAL128I) { - Preconditions.checkState(scalarType.isSetPrecision() - && scalarType.isSetScale()); - type = ScalarType.createDecimalV3Type(scalarType.getPrecision(), - scalarType.getScale()); - } else { - type = ScalarType.createType( - PrimitiveType.fromThrift(scalarType.getType())); - } - break; - } - case ARRAY: { - Preconditions.checkState(nodeIdx + 1 < typeDesc.getTypesSize()); - Pair childType = fromThrift(typeDesc, nodeIdx + 1); - type = new ArrayType(childType.first); - nodeIdx = childType.second; - break; - } - case MAP: { - Preconditions.checkState(nodeIdx + 2 < typeDesc.getTypesSize()); - Pair keyType = fromThrift(typeDesc, nodeIdx + 1); - Pair valueType = fromThrift(typeDesc, keyType.second); - type = new MapType(keyType.first, valueType.first); - nodeIdx = valueType.second; - break; - } + public void setKeyScale(int scale) { + this.keyScale = scale; + } - default: - throw new InternalException("Return type " + node.getType() + " is not supported now!"); + public void setValueScale(int scale) { + this.valueScale = scale; + } + + public int getKeyScale() { + return keyScale; } - return Pair.of(type, nodeIdx); - } - public static long getAddressAtOffset(long base, int offset) { - return base + 8L * offset; + public int getValueScale() { + return valueScale; + } } public static void copyMemory( @@ -339,9 +297,13 @@ public static Pair setReturnType(Type retType, Class setArgTypes(Type[] parameterTypes } else if (parameterTypes[finalI].isArrayType()) { ArrayType arrType = (ArrayType) parameterTypes[finalI]; inputArgTypes[i].setItemType(arrType.getItemType()); + if (arrType.getItemType().isDatetimeV2() || arrType.getItemType().isDecimalV3()) { + inputArgTypes[i].setPrecision(arrType.getItemType().getPrecision()); + inputArgTypes[i].setScale(((ScalarType) arrType.getItemType()).getScalarScale()); + } } else if (parameterTypes[finalI].isMapType()) { MapType mapType = (MapType) parameterTypes[finalI]; + Type keyType = mapType.getKeyType(); + Type valuType = mapType.getValueType(); inputArgTypes[i].setKeyType(mapType.getKeyType()); inputArgTypes[i].setValueType(mapType.getValueType()); + if (keyType.isDatetimeV2() || keyType.isDecimalV3()) { + inputArgTypes[i].setKeyScale(((ScalarType) keyType).getScalarScale()); + } + if (valuType.isDatetimeV2() || valuType.isDecimalV3()) { + inputArgTypes[i].setValueScale(((ScalarType) valuType).getScalarScale()); + } } if (res.length == 0) { return Pair.of(false, inputArgTypes); diff --git a/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/BaseExecutor.java b/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/BaseExecutor.java index 20f36866c8b168e..9c1cb17ba5c7f4a 100644 --- a/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/BaseExecutor.java +++ b/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/BaseExecutor.java @@ -24,6 +24,7 @@ import org.apache.doris.common.jni.utils.JNINativeMethod; import org.apache.doris.common.jni.utils.UdfUtils; import org.apache.doris.common.jni.utils.UdfUtils.JavaUdfDataType; +import org.apache.doris.thrift.TFunction; import org.apache.doris.thrift.TJavaUdfExecutorCtorParams; import com.esotericsoftware.reflectasm.MethodAccess; @@ -76,6 +77,7 @@ public abstract class BaseExecutor { protected JavaUdfDataType retType; protected Class[] argClass; protected MethodAccess methodAccess; + protected TFunction fn; /** * Create a UdfExecutor, using parameters from a serialized thrift object. Used @@ -95,11 +97,41 @@ public BaseExecutor(byte[] thriftParams) throws Exception { for (int i = 0; i < request.fn.arg_types.size(); ++i) { parameterTypes[i] = Type.fromThrift(request.fn.arg_types.get(i)); } + fn = request.fn; String jarFile = request.location; Type funcRetType = Type.fromThrift(request.fn.ret_type); init(request, jarFile, funcRetType, parameterTypes); } + public String debugString() { + String res = ""; + for (JavaUdfDataType type : argTypes) { + res = res + type.toString(); + if (type.getItemType() != null) { + res = res + " item: " + type.getItemType().toString() + " sql: " + type.getItemType().toSql(); + } + if (type.getKeyType() != null) { + res = res + " key: " + type.getKeyType().toString() + " sql: " + type.getKeyType().toSql(); + } + if (type.getValueType() != null) { + res = res + " key: " + type.getValueType().toString() + " sql: " + type.getValueType().toSql(); + } + } + res = res + " return type: " + retType.toString(); + if (retType.getItemType() != null) { + res = res + " item: " + retType.getItemType().toString() + " sql: " + retType.getItemType().toSql(); + } + if (retType.getKeyType() != null) { + res = res + " key: " + retType.getKeyType().toString() + " sql: " + retType.getKeyType().toSql(); + } + if (retType.getValueType() != null) { + res = res + " key: " + retType.getValueType().toString() + " sql: " + retType.getValueType().toSql(); + } + res = res + " methodAccess: " + methodAccess.toString(); + res = res + " fn.toString(): " + fn.toString(); + return res; + } + protected abstract void init(TJavaUdfExecutorCtorParams request, String jarPath, Type funcRetType, Type... parameterTypes) throws UdfRuntimeException; @@ -248,7 +280,6 @@ public void copyTupleBasicResult(Object obj, long row, Class retClass, UdfUtils.UNSAFE.putInt(null, offsetsAddr + 4L * row, Integer.parseUnsignedInt(String.valueOf(offset))); UdfUtils.copyMemory(bytes, UdfUtils.BYTE_ARRAY_OFFSET, null, outputBufferBase + offset - bytes.length, bytes.length); - updateOutputOffset(offset); break; } case ARRAY_TYPE: @@ -257,10 +288,6 @@ public void copyTupleBasicResult(Object obj, long row, Class retClass, } } - - protected void updateOutputOffset(long offset) { - } - public Object[] convertBasicArg(boolean isUdf, int argIdx, boolean isNullable, int rowStart, int rowEnd, long nullMapAddr, long columnAddr, long strOffsetAddr) { switch (argTypes[argIdx]) { @@ -442,7 +469,7 @@ public Object[] convertArrayArg(int argIdx, boolean isNullable, int rowStart, in public Object[] convertMapArg(PrimitiveType type, int argIdx, boolean isNullable, int rowStart, int rowEnd, long nullMapAddr, - long offsetsAddr, long nestedNullMapAddr, long dataAddr, long strOffsetAddr) { + long offsetsAddr, long nestedNullMapAddr, long dataAddr, long strOffsetAddr, int scale) { Object[] argument = (Object[]) Array.newInstance(ArrayList.class, rowEnd - rowStart); for (int row = rowStart; row < rowEnd; ++row) { long offsetStart = UdfUtils.UNSAFE.getLong(null, offsetsAddr + 8L * (row - 1)); @@ -533,19 +560,19 @@ public Object[] convertMapArg(PrimitiveType type, int argIdx, boolean isNullable case DECIMALV2: case DECIMAL128: { argument[row - rowStart] = UdfConvert - .convertArrayDecimalArg(argTypes[argIdx].getScale(), 16L, row, currentRowNum, + .convertArrayDecimalArg(scale, 16L, row, currentRowNum, offsetStart, isNullable, nullMapAddr, nestedNullMapAddr, dataAddr); break; } case DECIMAL32: { argument[row - rowStart] = UdfConvert - .convertArrayDecimalArg(argTypes[argIdx].getScale(), 4L, row, currentRowNum, + .convertArrayDecimalArg(scale, 4L, row, currentRowNum, offsetStart, isNullable, nullMapAddr, nestedNullMapAddr, dataAddr); break; } case DECIMAL64: { argument[row - rowStart] = UdfConvert - .convertArrayDecimalArg(argTypes[argIdx].getScale(), 8L, row, currentRowNum, + .convertArrayDecimalArg(scale, 8L, row, currentRowNum, offsetStart, isNullable, nullMapAddr, nestedNullMapAddr, dataAddr); break; } @@ -785,18 +812,18 @@ public void copyBatchBasicResultImpl(boolean isNullable, int numRows, Object[] r public void copyBatchArrayResultImpl(boolean isNullable, int numRows, Object[] result, long nullMapAddr, long offsetsAddr, long nestedNullMapAddr, long dataAddr, long strOffsetAddr, - PrimitiveType type) { + PrimitiveType type, int scale) { long hasPutElementNum = 0; for (int row = 0; row < numRows; ++row) { hasPutElementNum = copyTupleArrayResultImpl(hasPutElementNum, isNullable, row, result[row], nullMapAddr, - offsetsAddr, nestedNullMapAddr, dataAddr, strOffsetAddr, type); + offsetsAddr, nestedNullMapAddr, dataAddr, strOffsetAddr, type, scale); } } public long copyTupleArrayResultImpl(long hasPutElementNum, boolean isNullable, int row, Object result, long nullMapAddr, long offsetsAddr, long nestedNullMapAddr, long dataAddr, long strOffsetAddr, - PrimitiveType type) { + PrimitiveType type, int scale) { switch (type) { case BOOLEAN: { hasPutElementNum = UdfConvert @@ -886,21 +913,21 @@ public long copyTupleArrayResultImpl(long hasPutElementNum, boolean isNullable, } case DECIMAL32: { hasPutElementNum = UdfConvert - .copyBatchArrayDecimalV3Result(retType.getScale(), 4L, hasPutElementNum, isNullable, row, + .copyBatchArrayDecimalV3Result(scale, 4L, hasPutElementNum, isNullable, row, result, nullMapAddr, offsetsAddr, nestedNullMapAddr, dataAddr); break; } case DECIMAL64: { hasPutElementNum = UdfConvert - .copyBatchArrayDecimalV3Result(retType.getScale(), 8L, hasPutElementNum, isNullable, row, + .copyBatchArrayDecimalV3Result(scale, 8L, hasPutElementNum, isNullable, row, result, nullMapAddr, offsetsAddr, nestedNullMapAddr, dataAddr); break; } case DECIMAL128: { hasPutElementNum = UdfConvert - .copyBatchArrayDecimalV3Result(retType.getScale(), 16L, hasPutElementNum, isNullable, row, + .copyBatchArrayDecimalV3Result(scale, 16L, hasPutElementNum, isNullable, row, result, nullMapAddr, offsetsAddr, nestedNullMapAddr, dataAddr); break; diff --git a/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/UdafExecutor.java b/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/UdafExecutor.java index fa19ad32888d2f0..29ac4b272bacb53 100644 --- a/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/UdafExecutor.java +++ b/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/UdafExecutor.java @@ -48,7 +48,6 @@ public class UdafExecutor extends BaseExecutor { private static final Logger LOG = Logger.getLogger(UdafExecutor.class); - private long inputPlacesPtr; private HashMap allMethods; private HashMap stateObjMap; private Class retClass; @@ -88,11 +87,11 @@ public Object[] convertMapArguments(int argIdx, boolean isNullable, int rowStart PrimitiveType valueType = argTypes[argIdx].getValueType().getPrimitiveType(); Object[] keyCol = convertMapArg(keyType, argIdx, isNullable, rowStart, rowEnd, nullMapAddr, offsetsAddr, keyNestedNullMapAddr, keyDataAddr, - keyStrOffsetAddr); + keyStrOffsetAddr, argTypes[argIdx].getKeyScale()); Object[] valueCol = convertMapArg(valueType, argIdx, isNullable, rowStart, rowEnd, nullMapAddr, offsetsAddr, valueNestedNullMapAddr, valueDataAddr, - valueStrOffsetAddr); + valueStrOffsetAddr, argTypes[argIdx].getValueScale()); return buildHashMap(keyType, valueType, keyCol, valueCol); } @@ -126,6 +125,7 @@ public void addBatchSingle(int rowStart, int rowEnd, long placeAddr, Object[] co methodAccess.invoke(udf, addIndex, inputArgs); } } catch (Exception e) { + LOG.info("evaluate exception debug: " + debugString()); LOG.info("invoke add function meet some error: " + e.getCause().toString()); throw new UdfRuntimeException("UDAF failed to addBatchSingle: ", e); } @@ -158,6 +158,7 @@ public void addBatchPlaces(int rowStart, int rowEnd, long placeAddr, int offset, methodAccess.invoke(udf, addIndex, inputArgs); } } catch (Exception e) { + LOG.info("evaluate exception debug: " + debugString()); LOG.info("invoke add function meet some error: " + Arrays.toString(e.getStackTrace())); throw new UdfRuntimeException("UDAF failed to addBatchPlaces: ", e); } @@ -202,6 +203,7 @@ public byte[] serialize(long place) throws UdfRuntimeException { allMethods.get(UDAF_SERIALIZE_FUNCTION).invoke(udf, args); return baos.toByteArray(); } catch (Exception e) { + LOG.info("evaluate exception debug: " + debugString()); LOG.warn("invoke serialize function meet some error: " + e.getCause().toString()); throw new UdfRuntimeException("UDAF failed to serialize: ", e); } @@ -219,6 +221,7 @@ public void reset(long place) throws UdfRuntimeException { } allMethods.get(UDAF_RESET_FUNCTION).invoke(udf, args); } catch (Exception e) { + LOG.info("evaluate exception debug: " + debugString()); LOG.warn("invoke reset function meet some error: " + e.getCause().toString()); throw new UdfRuntimeException("UDAF failed to reset: ", e); } @@ -247,6 +250,7 @@ public void merge(long place, byte[] data) throws UdfRuntimeException { } allMethods.get(UDAF_MERGE_FUNCTION).invoke(udf, args); } catch (Exception e) { + LOG.info("evaluate exception debug: " + debugString()); LOG.warn("invoke merge function meet some error: " + e.getCause().toString()); throw new UdfRuntimeException("UDAF failed to merge: ", e); } @@ -263,6 +267,7 @@ public Object getValue(long place) throws UdfRuntimeException { } return allMethods.get(UDAF_RESULT_FUNCTION).invoke(udf, stateObjMap.get((Long) place)); } catch (Exception e) { + LOG.info("evaluate exception debug: " + debugString()); LOG.warn("invoke getValue function meet some error: " + e.getCause().toString()); throw new UdfRuntimeException("UDAF failed to result", e); } @@ -298,12 +303,13 @@ public void copyTupleArrayResult(long hasPutElementNum, boolean isNullable, int UdfUtils.UNSAFE.putByte(nullMapAddr + row, (byte) 0); } copyTupleArrayResultImpl(hasPutElementNum, isNullable, row, result, nullMapAddr, offsetsAddr, nestedNullMapAddr, - dataAddr, strOffsetAddr, retType.getItemType().getPrimitiveType()); + dataAddr, strOffsetAddr, retType.getItemType().getPrimitiveType(), retType.getScale()); } public void copyTupleMapResult(long hasPutElementNum, boolean isNullable, int row, Object result, long nullMapAddr, long offsetsAddr, - long keyNsestedNullMapAddr, long keyDataAddr, long keyStrOffsetAddr, + long keyNsestedNullMapAddr, long keyDataAddr, + long keyStrOffsetAddr, long valueNsestedNullMapAddr, long valueDataAddr, long valueStrOffsetAddr) throws UdfRuntimeException { if (nullMapAddr > 0) { UdfUtils.UNSAFE.putByte(nullMapAddr + row, (byte) 0); @@ -317,16 +323,15 @@ public void copyTupleMapResult(long hasPutElementNum, boolean isNullable, int ro buildArrayListFromHashMap(resultArr, keyType, valueType, keyCol, valueCol); copyTupleArrayResultImpl(hasPutElementNum, isNullable, row, valueCol[0], nullMapAddr, offsetsAddr, - valueNsestedNullMapAddr, valueDataAddr, valueStrOffsetAddr, valueType); + valueNsestedNullMapAddr, valueDataAddr, valueStrOffsetAddr, valueType, retType.getKeyScale()); copyTupleArrayResultImpl(hasPutElementNum, isNullable, row, keyCol[0], nullMapAddr, offsetsAddr, - keyNsestedNullMapAddr, keyDataAddr, keyStrOffsetAddr, keyType); + keyNsestedNullMapAddr, keyDataAddr, keyStrOffsetAddr, keyType, retType.getValueScale()); } @Override protected void init(TJavaUdfExecutorCtorParams request, String jarPath, Type funcRetType, Type... parameterTypes) throws UdfRuntimeException { String className = request.fn.aggregate_fn.symbol; - inputPlacesPtr = request.input_places_ptr; allMethods = new HashMap<>(); stateObjMap = new HashMap<>(); diff --git a/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/UdfExecutor.java b/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/UdfExecutor.java index a77b441b67d997a..f0319a5304e6658 100644 --- a/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/UdfExecutor.java +++ b/fe/be-java-extensions/java-udf/src/main/java/org/apache/doris/udf/UdfExecutor.java @@ -44,15 +44,6 @@ public class UdfExecutor extends BaseExecutor { // setup by init() and cleared by close() private Method method; - // Pre-constructed input objects for the UDF. This minimizes object creation - // overhead - // as these objects are reused across calls to evaluate(). - private Object[] inputObjects; - - private long outputOffset; - private long rowIdx; - - private long batchSizePtr; private int evaluateIndex; /** @@ -92,10 +83,10 @@ public Object[] convertMapArguments(int argIdx, boolean isNullable, int numRows, PrimitiveType valueType = argTypes[argIdx].getValueType().getPrimitiveType(); Object[] keyCol = convertMapArg(keyType, argIdx, isNullable, 0, numRows, nullMapAddr, offsetsAddr, keyNestedNullMapAddr, keyDataAddr, - keyStrOffsetAddr); + keyStrOffsetAddr, argTypes[argIdx].getKeyScale()); Object[] valueCol = convertMapArg(valueType, argIdx, isNullable, 0, numRows, nullMapAddr, offsetsAddr, valueNestedNullMapAddr, valueDataAddr, - valueStrOffsetAddr); + valueStrOffsetAddr, argTypes[argIdx].getValueScale()); return buildHashMap(keyType, valueType, keyCol, valueCol); } @@ -115,6 +106,7 @@ public Object[] evaluate(int numRows, Object[] column) throws UdfRuntimeExceptio } return result; } catch (Exception e) { + LOG.info("evaluate exception: " + debugString()); LOG.info("evaluate(int numRows, Object[] column) Exception: " + e.toString()); throw new UdfRuntimeException("UDF failed to evaluate", e); } @@ -130,7 +122,7 @@ public void copyBatchArrayResult(boolean isNullable, int numRows, Object[] resul Preconditions.checkState(result.length == numRows, "copyBatchArrayResult result size should equal;"); copyBatchArrayResultImpl(isNullable, numRows, result, nullMapAddr, offsetsAddr, nestedNullMapAddr, dataAddr, - strOffsetAddr, retType.getItemType().getPrimitiveType()); + strOffsetAddr, retType.getItemType().getPrimitiveType(), retType.getScale()); } public void copyBatchMapResult(boolean isNullable, int numRows, Object[] result, long nullMapAddr, @@ -146,10 +138,10 @@ public void copyBatchMapResult(boolean isNullable, int numRows, Object[] result, copyBatchArrayResultImpl(isNullable, numRows, valueCol, nullMapAddr, offsetsAddr, valueNsestedNullMapAddr, valueDataAddr, - valueStrOffsetAddr, valueType); + valueStrOffsetAddr, valueType, retType.getKeyScale()); copyBatchArrayResultImpl(isNullable, numRows, keyCol, nullMapAddr, offsetsAddr, keyNsestedNullMapAddr, keyDataAddr, - keyStrOffsetAddr, keyType); + keyStrOffsetAddr, keyType, retType.getValueScale()); } /** @@ -167,21 +159,12 @@ public Method getMethod() { return method; } - - @Override - protected void updateOutputOffset(long offset) { - outputOffset = offset; - } - // Preallocate the input objects that will be passed to the underlying UDF. // These objects are allocated once and reused across calls to evaluate() @Override protected void init(TJavaUdfExecutorCtorParams request, String jarPath, Type funcRetType, Type... parameterTypes) throws UdfRuntimeException { String className = request.fn.scalar_fn.symbol; - batchSizePtr = request.batch_size_ptr; - outputOffset = 0L; - rowIdx = 0L; ArrayList signatures = Lists.newArrayList(); try { LOG.debug("Loading UDF '" + className + "' from " + jarPath); diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java index d35750ee6989eac..50fddda0f3d0504 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java @@ -169,6 +169,8 @@ public static ImmutableSetMultimap getImplicitCast builder.put(TINYINT, DECIMAL128); builder.put(TINYINT, VARCHAR); builder.put(TINYINT, STRING); + builder.put(TINYINT, TIME); + builder.put(TINYINT, TIMEV2); // Smallint builder.put(SMALLINT, BOOLEAN); builder.put(SMALLINT, TINYINT); @@ -188,6 +190,8 @@ public static ImmutableSetMultimap getImplicitCast builder.put(SMALLINT, DECIMAL128); builder.put(SMALLINT, VARCHAR); builder.put(SMALLINT, STRING); + builder.put(SMALLINT, TIME); + builder.put(SMALLINT, TIMEV2); // Int builder.put(INT, BOOLEAN); builder.put(INT, TINYINT); @@ -207,6 +211,8 @@ public static ImmutableSetMultimap getImplicitCast builder.put(INT, DECIMAL128); builder.put(INT, VARCHAR); builder.put(INT, STRING); + builder.put(INT, TIME); + builder.put(INT, TIMEV2); // Bigint builder.put(BIGINT, BOOLEAN); builder.put(BIGINT, TINYINT); @@ -226,6 +232,8 @@ public static ImmutableSetMultimap getImplicitCast builder.put(BIGINT, DECIMAL128); builder.put(BIGINT, VARCHAR); builder.put(BIGINT, STRING); + builder.put(BIGINT, TIME); + builder.put(BIGINT, TIMEV2); // Largeint builder.put(LARGEINT, BOOLEAN); builder.put(LARGEINT, TINYINT); @@ -245,6 +253,8 @@ public static ImmutableSetMultimap getImplicitCast builder.put(LARGEINT, DECIMAL128); builder.put(LARGEINT, VARCHAR); builder.put(LARGEINT, STRING); + builder.put(LARGEINT, TIME); + builder.put(LARGEINT, TIMEV2); // Float builder.put(FLOAT, BOOLEAN); builder.put(FLOAT, TINYINT); @@ -264,6 +274,8 @@ public static ImmutableSetMultimap getImplicitCast builder.put(FLOAT, DECIMAL128); builder.put(FLOAT, VARCHAR); builder.put(FLOAT, STRING); + builder.put(FLOAT, TIME); + builder.put(FLOAT, TIMEV2); // Double builder.put(DOUBLE, BOOLEAN); builder.put(DOUBLE, TINYINT); @@ -283,6 +295,8 @@ public static ImmutableSetMultimap getImplicitCast builder.put(DOUBLE, DECIMAL128); builder.put(DOUBLE, VARCHAR); builder.put(DOUBLE, STRING); + builder.put(DOUBLE, TIME); + builder.put(DOUBLE, TIMEV2); // Date builder.put(DATE, BOOLEAN); builder.put(DATE, TINYINT); @@ -379,6 +393,8 @@ public static ImmutableSetMultimap getImplicitCast builder.put(CHAR, DECIMAL128); builder.put(CHAR, VARCHAR); builder.put(CHAR, STRING); + builder.put(CHAR, TIME); + builder.put(CHAR, TIMEV2); // Varchar builder.put(VARCHAR, BOOLEAN); builder.put(VARCHAR, TINYINT); @@ -399,8 +415,10 @@ public static ImmutableSetMultimap getImplicitCast builder.put(VARCHAR, VARCHAR); builder.put(VARCHAR, JSONB); builder.put(VARCHAR, STRING); + builder.put(VARCHAR, TIME); + builder.put(VARCHAR, TIMEV2); - // Varchar + // String builder.put(STRING, BOOLEAN); builder.put(STRING, TINYINT); builder.put(STRING, SMALLINT); @@ -420,6 +438,8 @@ public static ImmutableSetMultimap getImplicitCast builder.put(STRING, VARCHAR); builder.put(STRING, JSONB); builder.put(STRING, STRING); + builder.put(STRING, TIME); + builder.put(STRING, TIMEV2); // DecimalV2 builder.put(DECIMALV2, BOOLEAN); 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 9da22b20da59320..49b2d8547a23890 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 @@ -1554,7 +1554,21 @@ public class Config extends ConfigBase { public static boolean enable_pipeline_load = false; @ConfField - public static int scheduler_job_task_max_num = 10; + public static int scheduler_job_task_max_saved_count = 10; + + /** + * The number of async tasks that can be queued. @See TaskDisruptor + * if consumer is slow, the queue will be full, and the producer will be blocked. + */ + @ConfField + public static int async_task_queen_size = 1024; + + /** + * The number of threads used to consume async tasks. @See TaskDisruptor + * if we have a lot of async tasks, we need more threads to consume them. Sure, it's depends on the cpu cores. + */ + @ConfField + public static int async_task_consumer_thread_num = 10; // enable_workload_group should be immutable and temporarily set to mutable during the development test phase @ConfField(mutable = true, varType = VariableAnnotation.EXPERIMENTAL) @@ -1770,7 +1784,7 @@ public class Config extends ConfigBase { * For external schema cache and hive meta cache. */ @ConfField(mutable = false, masterOnly = false) - public static long external_cache_expire_time_minutes_after_access = 24 * 60; // 1 day + public static long external_cache_expire_time_minutes_after_access = 10; // 10 mins /** * Github workflow test type, for setting some session variables @@ -2122,4 +2136,11 @@ public class Config extends ConfigBase { @ConfField public static int table_stats_health_threshold = 80; + @ConfField(mutable = true, masterOnly = false, description = { + "查询information_schema.metadata_name_ids表时,获取一个数据库中所有表用的时间", + "When querying the information_schema.metadata_name_ids table," + + " the time used to obtain all tables in one database" + }) + public static long query_metadata_name_ids_timeout = 3; + } diff --git a/fe/fe-core/pom.xml b/fe/fe-core/pom.xml index 40afee2bcd86b1e..26e49d276e6432c 100644 --- a/fe/fe-core/pom.xml +++ b/fe/fe-core/pom.xml @@ -448,12 +448,12 @@ under the License. net.java.dev.jna jna - 5.5.0 + 5.13.0 net.java.dev.jna jna-platform - 5.5.0 + 5.13.0 software.amazon.awssdk @@ -574,6 +574,12 @@ under the License. ${paimon.version} + + io.delta + delta-standalone_2.12 + ${delta.version} + + software.amazon.awssdk glue diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 index 95c21ff35224f64..ba81e1ba7671acf 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 @@ -82,6 +82,8 @@ COMMA: ','; DOT: '.'; LEFT_BRACKET: '['; RIGHT_BRACKET: ']'; +LEFT_BRACE: '{'; +RIGHT_BRACE: '}'; // TODO: add a doc to list reserved words @@ -89,320 +91,445 @@ RIGHT_BRACKET: ']'; // Start of the keywords list //============================ //--DORIS-KEYWORD-LIST-START +ACCOUNT_LOCK: 'ACCOUNT_LOCK'; +ACCOUNT_UNLOCK: 'ACCOUNT_UNLOCK'; ADD: 'ADD'; ADDDATE:'ADDDATE'; +ADMIN: 'ADMIN'; AFTER: 'AFTER'; +AGG_STATE: 'AGG_STATE'; +AGGREGATE: 'AGGREGATE'; +ALIAS: 'ALIAS'; ALL: 'ALL'; ALTER: 'ALTER'; ANALYZE: 'ANALYZE'; ANALYZED: 'ANALYZED'; AND: 'AND'; ANTI: 'ANTI'; -ANY: 'ANY'; -ARCHIVE: 'ARCHIVE'; +APPEND: 'APPEND'; ARRAY: 'ARRAY'; AS: 'AS'; ASC: 'ASC'; AT: 'AT'; -AUTHORIZATION: 'AUTHORIZATION'; -AVG: 'AVG'; +AUTHORS: 'AUTHORS'; +AUTO: 'AUTO'; +AUTO_INCREMENT: 'AUTO_INCREMENT'; +BACKEND: 'BACKEND'; +BACKENDS: 'BACKENDS'; +BACKUP: 'BACKUP'; +BEGIN: 'BEGIN'; BETWEEN: 'BETWEEN'; +BIGINT: 'BIGINT'; +BIN: 'BIN'; +BINLOG: 'BINLOG'; BITAND: 'BITAND'; +BITMAP: 'BITMAP'; +BITMAP_UNION: 'BITMAP_UNION'; BITOR: 'BITOR'; BITXOR: 'BITXOR'; -BOTH: 'BOTH'; -BUCKET: 'BUCKET'; +BLOB: 'BLOB'; +BOOLEAN: 'BOOLEAN'; +BRIEF: 'BRIEF'; +BROKER: 'BROKER'; BUCKETS: 'BUCKETS'; +BUILD: 'BUILD'; +BUILTIN: 'BUILTIN'; BY: 'BY'; -CACHE: 'CACHE'; -CASCADE: 'CASCADE'; +CACHED: 'CACHED'; +CANCEL: 'CANCEL'; CASE: 'CASE'; CAST: 'CAST'; CATALOG: 'CATALOG'; CATALOGS: 'CATALOGS'; -CHANGE: 'CHANGE'; +CHAIN: 'CHAIN'; +CHAR: 'CHAR' | 'CHARACTER'; +CHARSET: 'CHARSET'; CHECK: 'CHECK'; -CLEAR: 'CLEAR'; +CLEAN: 'CLEAN'; CLUSTER: 'CLUSTER'; -CLUSTERED: 'CLUSTERED'; -CODEGEN: 'CODEGEN'; +CLUSTERS: 'CLUSTERS'; COLLATE: 'COLLATE'; -COLLECTION: 'COLLECTION'; +COLLATION: 'COLLATION'; COLUMN: 'COLUMN'; COLUMNS: 'COLUMNS'; COMMENT: 'COMMENT'; COMMIT: 'COMMIT'; +COMMITTED: 'COMMITTED'; COMPACT: 'COMPACT'; -COMPACTIONS: 'COMPACTIONS'; -COMPUTE: 'COMPUTE'; -CONCATENATE: 'CONCATENATE'; -CONSTRAINT: 'CONSTRAINT'; -COST: 'COST'; +COMPLETE: 'COMPLETE'; +CONFIG: 'CONFIG'; +CONNECTION: 'CONNECTION'; +CONNECTION_ID: 'CONNECTION_ID'; +CONSISTENT: 'CONSISTENT'; +CONVERT: 'CONVERT'; +COPY: 'COPY'; +COUNT: 'COUNT'; CREATE: 'CREATE'; +CREATION: 'CREATION'; +CRON: 'CRON'; CROSS: 'CROSS'; CUBE: 'CUBE'; CURRENT: 'CURRENT'; -CURRENT_DATE: 'CURRENT_DATE'; -CURRENT_TIME: 'CURRENT_TIME'; +CURRENT_CATALOG: 'CURRENT_CATALOG'; CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP'; CURRENT_USER: 'CURRENT_USER'; -DAY: 'DAY'; -DAYS_ADD: 'DAYS_ADD'; -DAYS_SUB: 'DAYS_SUB'; DATA: 'DATA'; DATABASE: 'DATABASE'; DATABASES: 'DATABASES'; DATE: 'DATE'; -DATEADD: 'DATEADD'; -DATEDIFF: 'DATEDIFF'; DATE_ADD: 'DATE_ADD'; -DATE_SUB: 'DATE_SUB'; +DATE_CEIL: 'DATE_CEIL'; DATE_DIFF: 'DATE_DIFF'; -DBPROPERTIES: 'DBPROPERTIES'; +DATE_FLOOR: 'DATE_FLOOR'; +DATE_SUB: 'DATE_SUB'; +DATEADD: 'DATEADD'; +DATEDIFF: 'DATEDIFF'; +DATETIME: 'DATETIME'; +DATETIMEV2: 'DATETIMEV2'; +DATEV2: 'DATEV2'; +DAY: 'DAY'; +DAYS_ADD: 'DAYS_ADD'; +DAYS_SUB: 'DAYS_SUB'; +DECIMAL: 'DECIMAL'; +DECIMALV3: 'DECIMALV3'; +DECOMMISSION: 'DECOMMISSION'; DEFAULT: 'DEFAULT'; -DEFINED: 'DEFINED'; +DEFERRED: 'DEFERRED'; DELETE: 'DELETE'; -DELIMITED: 'DELIMITED'; +DEMAND: 'DEMAND'; DESC: 'DESC'; DESCRIBE: 'DESCRIBE'; -DFS: 'DFS'; -DIRECTORIES: 'DIRECTORIES'; -DIRECTORY: 'DIRECTORY'; +DIAGNOSE: 'DIAGNOSE'; +DISK: 'DISK'; DISTINCT: 'DISTINCT'; -DISTRIBUTE: 'DISTRIBUTE'; +DISTINCTPC: 'DISTINCTPC'; +DISTINCTPCSA: 'DISTINCTPCSA'; +DISTRIBUTED: 'DISTRIBUTED'; +DISTRIBUTION: 'DISTRIBUTION'; DIV: 'DIV'; +DO: 'DO'; +DOUBLE: 'DOUBLE'; DROP: 'DROP'; +DROPP: 'DROPP'; +DUPLICATE: 'DUPLICATE'; +DYNAMIC: 'DYNAMIC'; ELSE: 'ELSE'; +ENABLE: 'ENABLE'; +ENCRYPTKEY: 'ENCRYPTKEY'; +ENCRYPTKEYS: 'ENCRYPTKEYS'; END: 'END'; -ESCAPE: 'ESCAPE'; -ESCAPED: 'ESCAPED'; +ENDS: 'ENDS'; +ENGINE: 'ENGINE'; +ENGINES: 'ENGINES'; +ENTER: 'ENTER'; +ERRORS: 'ERRORS'; +EVENTS: 'EVENTS'; +EVERY: 'EVERY'; EXCEPT: 'EXCEPT'; -EXCHANGE: 'EXCHANGE'; +EXCLUDE: 'EXCLUDE'; +EXECUTE: 'EXECUTE'; EXISTS: 'EXISTS'; +EXPIRED: 'EXPIRED'; EXPLAIN: 'EXPLAIN'; EXPORT: 'EXPORT'; EXTENDED: 'EXTENDED'; EXTERNAL: 'EXTERNAL'; EXTRACT: 'EXTRACT'; +FAILED_LOGIN_ATTEMPTS: 'FAILED_LOGIN_ATTEMPTS'; FALSE: 'FALSE'; -FETCH: 'FETCH'; +FAST: 'FAST'; +FEATURE: 'FEATURE'; FIELDS: 'FIELDS'; +FILE: 'FILE'; FILTER: 'FILTER'; -FILEFORMAT: 'FILEFORMAT'; FIRST: 'FIRST'; +FLOAT: 'FLOAT'; +FOLLOWER: 'FOLLOWER'; FOLLOWING: 'FOLLOWING'; FOR: 'FOR'; -FOREIGN: 'FOREIGN'; +FORCE: 'FORCE'; FORMAT: 'FORMAT'; -FORMATTED: 'FORMATTED'; +FREE: 'FREE'; FROM: 'FROM'; +FRONTEND: 'FRONTEND'; +FRONTENDS: 'FRONTENDS'; FULL: 'FULL'; FUNCTION: 'FUNCTION'; FUNCTIONS: 'FUNCTIONS'; GLOBAL: 'GLOBAL'; GRANT: 'GRANT'; +GRANTS: 'GRANTS'; GRAPH: 'GRAPH'; GROUP: 'GROUP'; GROUPING: 'GROUPING'; +GROUPS: 'GROUPS'; +HASH: 'HASH'; HAVING: 'HAVING'; +HDFS: 'HDFS'; +HELP: 'HELP'; +HISTOGRAM: 'HISTOGRAM'; +HLL: 'HLL'; +HLL_UNION: 'HLL_UNION'; +HOSTNAME: 'HOSTNAME'; HOUR: 'HOUR'; +HUB: 'HUB'; +IDENTIFIED: 'IDENTIFIED'; IF: 'IF'; IGNORE: 'IGNORE'; -IMPORT: 'IMPORT'; +IMMEDIATE: 'IMMEDIATE'; IN: 'IN'; +INCREMENTAL: 'INCREMENTAL'; INDEX: 'INDEX'; INDEXES: 'INDEXES'; +INFILE: 'INFILE'; INNER: 'INNER'; -INPATH: 'INPATH'; -INPUTFORMAT: 'INPUTFORMAT'; INSERT: 'INSERT'; +INSTALL: 'INSTALL'; +INT: 'INT'; +INTEGER: 'INTEGER'; +INTERMEDIATE: 'INTERMEDIATE'; INTERSECT: 'INTERSECT'; INTERVAL: 'INTERVAL'; INTO: 'INTO'; +INVERTED: 'INVERTED'; IS: 'IS'; -ISNULL: 'ISNULL'; -IS_NULL_PRED: 'IS_NULL_PRED'; IS_NOT_NULL_PRED: 'IS_NOT_NULL_PRED'; -ITEMS: 'ITEMS'; +IS_NULL_PRED: 'IS_NULL_PRED'; +ISNULL: 'ISNULL'; +ISOLATION: 'ISOLATION'; +JOB: 'JOB'; +JOBS: 'JOBS'; JOIN: 'JOIN'; +JSON: 'JSON'; +JSONB: 'JSONB'; +KEY: 'KEY'; KEYS: 'KEYS'; +KILL: 'KILL'; LABEL: 'LABEL'; +LARGEINT: 'LARGEINT'; LAST: 'LAST'; LATERAL: 'LATERAL'; -LAZY: 'LAZY'; -LEADING: 'LEADING'; +LDAP: 'LDAP'; +LDAP_ADMIN_PASSWORD: 'LDAP_ADMIN_PASSWORD'; LEFT: 'LEFT'; +LESS: 'LESS'; +LEVEL: 'LEVEL'; LIKE: 'LIKE'; -ILIKE: 'ILIKE'; LIMIT: 'LIMIT'; -OFFSET: 'OFFSET'; LINES: 'LINES'; +LINK: 'LINK'; LIST: 'LIST'; LOAD: 'LOAD'; LOCAL: 'LOCAL'; LOCATION: 'LOCATION'; LOCK: 'LOCK'; -LOCKS: 'LOCKS'; LOGICAL: 'LOGICAL'; -MACRO: 'MACRO'; +LOW_PRIORITY: 'LOW_PRIORITY'; MAP: 'MAP'; MATCH: 'MATCH'; -MATCH_ANY: 'MATCH_ANY'; MATCH_ALL: 'MATCH_ALL'; +MATCH_ANY: 'MATCH_ANY'; +MATCH_ELEMENT_EQ: 'ELEMENT_EQ'; +MATCH_ELEMENT_GE: 'ELEMENT_GE'; +MATCH_ELEMENT_GT: 'ELEMENT_GT'; +MATCH_ELEMENT_LE: 'ELEMENT_LE'; +MATCH_ELEMENT_LT: 'ELEMENT_LT'; MATCH_PHRASE: 'MATCH_PHRASE'; -MATCHED: 'MATCHED'; +MATERIALIZED: 'MATERIALIZED'; +MAX: 'MAX'; +MAX_VALUE: 'MAXVALUE'; MEMO:'MEMO'; MERGE: 'MERGE'; +MIGRATE: 'MIGRATE'; +MIGRATIONS: 'MIGRATIONS'; +MIN: 'MIN'; +MINUS: 'MINUS'; MINUTE: 'MINUTE'; +MODIFY: 'MODIFY'; MONTH: 'MONTH'; -MSCK: 'MSCK'; -NAMESPACE: 'NAMESPACE'; -NAMESPACES: 'NAMESPACES'; +MTMV: 'MTMV'; +NAME: 'NAME'; +NAMES: 'NAMES'; NATURAL: 'NATURAL'; +NEGATIVE: 'NEGATIVE'; +NEVER: 'NEVER'; +NEXT: 'NEXT'; +NGRAM_BF: 'NGRAM_BF'; NO: 'NO'; -NOT: 'NOT' | '!'; +NOT: 'NOT'; NULL: 'NULL'; NULLS: 'NULLS'; +OBSERVER: 'OBSERVER'; OF: 'OF'; +OFFSET: 'OFFSET'; ON: 'ON'; ONLY: 'ONLY'; +OPEN: 'OPEN'; OPTIMIZED: 'OPTIMIZED'; -OPTION: 'OPTION'; -OPTIONS: 'OPTIONS'; OR: 'OR'; ORDER: 'ORDER'; -OUT: 'OUT'; OUTER: 'OUTER'; OUTFILE: 'OUTFILE'; -OUTPUTFORMAT: 'OUTPUTFORMAT'; OVER: 'OVER'; -OVERLAPS: 'OVERLAPS'; -OVERLAY: 'OVERLAY'; OVERWRITE: 'OVERWRITE'; +PARAMETER: 'PARAMETER'; PARSED: 'PARSED'; PARTITION: 'PARTITION'; -PARTITIONED: 'PARTITIONED'; PARTITIONS: 'PARTITIONS'; -PERCENTILE_CONT: 'PERCENTILE_CONT'; -PERCENTLIT: 'PERCENT'; +PASSWORD: 'PASSWORD'; +PASSWORD_EXPIRE: 'PASSWORD_EXPIRE'; +PASSWORD_HISTORY: 'PASSWORD_HISTORY'; +PASSWORD_LOCK_TIME: 'PASSWORD_LOCK_TIME'; +PASSWORD_REUSE: 'PASSWORD_REUSE'; +PATH: 'PATH'; +PAUSE: 'PAUSE'; +PERCENT: 'PERCENT'; +PERIOD: 'PERIOD'; PERMISSIVE: 'PERMISSIVE'; PHYSICAL: 'PHYSICAL'; -PIVOT: 'PIVOT'; -PLACING: 'PLACING'; PLAN: 'PLAN'; +PLUGIN: 'PLUGIN'; +PLUGINS: 'PLUGINS'; POLICY: 'POLICY'; -POSITION: 'POSITION'; PRECEDING: 'PRECEDING'; -PRIMARY: 'PRIMARY'; -PRINCIPALS: 'PRINCIPALS'; +PREPARE: 'PREPARE'; +PROC: 'PROC'; +PROCEDURE: 'PROCEDURE'; +PROCESSLIST: 'PROCESSLIST'; +PROFILE: 'PROFILE'; PROPERTIES: 'PROPERTIES'; -PURGE: 'PURGE'; +PROPERTY: 'PROPERTY'; +QUANTILE_STATE: 'QUANTILE_STATE'; +QUANTILE_UNION: 'QUANTILE_UNION'; QUERY: 'QUERY'; +QUOTA: 'QUOTA'; +RANDOM: 'RANDOM'; RANGE: 'RANGE'; -RECORDREADER: 'RECORDREADER'; -RECORDWRITER: 'RECORDWRITER'; +READ: 'READ'; +REAL: 'REAL'; +REBALANCE: 'REBALANCE'; RECOVER: 'RECOVER'; -REDUCE: 'REDUCE'; -REFERENCES: 'REFERENCES'; +RECYCLE: 'RECYCLE'; REFRESH: 'REFRESH'; REGEXP: 'REGEXP'; +RELEASE: 'RELEASE'; RENAME: 'RENAME'; REPAIR: 'REPAIR'; REPEATABLE: 'REPEATABLE'; REPLACE: 'REPLACE'; -RESET: 'RESET'; -RESPECT: 'RESPECT'; -RESTRICT: 'RESTRICT'; +REPLACE_IF_NOT_NULL: 'REPLACE_IF_NOT_NULL'; +REPLICA: 'REPLICA'; +REPOSITORIES: 'REPOSITORIES'; +REPOSITORY: 'REPOSITORY'; +RESOURCE: 'RESOURCE'; +RESOURCES: 'RESOURCES'; +RESTORE: 'RESTORE'; RESTRICTIVE: 'RESTRICTIVE'; +RESUME: 'RESUME'; +RETURNS: 'RETURNS'; REVOKE: 'REVOKE'; REWRITTEN: 'REWRITTEN'; RIGHT: 'RIGHT'; -// original optimizer only support REGEXP, the new optimizer should be consistent with it RLIKE: 'RLIKE'; ROLE: 'ROLE'; ROLES: 'ROLES'; ROLLBACK: 'ROLLBACK'; ROLLUP: 'ROLLUP'; +ROUTINE: 'ROUTINE'; ROW: 'ROW'; ROWS: 'ROWS'; -SECOND: 'SECOND'; +S3: 'S3'; +SAMPLE: 'SAMPLE'; +SCHEDULER: 'SCHEDULER'; SCHEMA: 'SCHEMA'; SCHEMAS: 'SCHEMAS'; +SECOND: 'SECOND'; SELECT: 'SELECT'; SEMI: 'SEMI'; -SEPARATED: 'SEPARATED'; -SERDE: 'SERDE'; -SERDEPROPERTIES: 'SERDEPROPERTIES'; +SERIALIZABLE: 'SERIALIZABLE'; SESSION: 'SESSION'; -SESSION_USER: 'SESSION_USER'; SET: 'SET'; -SETMINUS: 'MINUS'; SETS: 'SETS'; SHAPE: 'SHAPE'; SHOW: 'SHOW'; -SKEWED: 'SKEWED'; -SOME: 'SOME'; -SORT: 'SORT'; -SORTED: 'SORTED'; +SIGNED: 'SIGNED'; +SKEW: 'SKEW'; +SMALLINT: 'SMALLINT'; +SNAPSHOT: 'SNAPSHOT'; +SONAME: 'SONAME'; +SPLIT: 'SPLIT'; +SQL_BLOCK_RULE: 'SQL_BLOCK_RULE'; START: 'START'; -STATISTICS: 'STATISTICS'; +STARTS: 'STARTS'; +STATS: 'STATS'; +STATUS: 'STATUS'; +STOP: 'STOP'; STORAGE: 'STORAGE'; -STORED: 'STORED'; -STRATIFY: 'STRATIFY'; +STREAM: 'STREAM'; +STREAMING: 'STREAMING'; +STRING: 'STRING'; STRUCT: 'STRUCT'; SUBDATE: 'SUBDATE'; +SUM: 'SUM'; +SUPERUSER: 'SUPERUSER'; +SWITCH: 'SWITCH'; SYNC: 'SYNC'; -SYSTEM_TIME: 'SYSTEM_TIME'; -SYSTEM_VERSION: 'SYSTEM_VERSION'; +SYSTEM: 'SYSTEM'; TABLE: 'TABLE'; TABLES: 'TABLES'; -TABLET: 'TABLET'; TABLESAMPLE: 'TABLESAMPLE'; -TBLPROPERTIES: 'TBLPROPERTIES'; -TEMPORARY: 'TEMPORARY' | 'TEMP'; +TABLET: 'TABLET'; +TABLETS: 'TABLETS'; +TASK: 'TASK'; +TASKS: 'TASKS'; +TEMPORARY: 'TEMPORARY'; TERMINATED: 'TERMINATED'; +TEXT: 'TEXT'; +THAN: 'THAN'; THEN: 'THEN'; TIME: 'TIME'; TIMESTAMP: 'TIMESTAMP'; TIMESTAMPADD: 'TIMESTAMPADD'; TIMESTAMPDIFF: 'TIMESTAMPDIFF'; +TINYINT: 'TINYINT'; TO: 'TO'; -TOUCH: 'TOUCH'; -TRAILING: 'TRAILING'; TRANSACTION: 'TRANSACTION'; -TRANSACTIONS: 'TRANSACTIONS'; -TRANSFORM: 'TRANSFORM'; +TRASH: 'TRASH'; +TRIGGERS: 'TRIGGERS'; TRIM: 'TRIM'; TRUE: 'TRUE'; TRUNCATE: 'TRUNCATE'; -TRY_CAST: 'TRY_CAST'; TYPE: 'TYPE'; -UNARCHIVE: 'UNARCHIVE'; +TYPECAST: 'TYPE_CAST'; +TYPES: 'TYPES'; UNBOUNDED: 'UNBOUNDED'; -UNCACHE: 'UNCACHE'; +UNCOMMITTED: 'UNCOMMITTED'; +UNINSTALL: 'UNINSTALL'; UNION: 'UNION'; UNIQUE: 'UNIQUE'; -UNKNOWN: 'UNKNOWN'; UNLOCK: 'UNLOCK'; -UNSET: 'UNSET'; +UNSIGNED: 'UNSIGNED'; UPDATE: 'UPDATE'; USE: 'USE'; USER: 'USER'; USING: 'USING'; +VALUE: 'VALUE'; VALUES: 'VALUES'; +VARCHAR: 'VARCHAR'; +VARIABLES: 'VARIABLES'; VERBOSE: 'VERBOSE'; VERSION: 'VERSION'; VIEW: 'VIEW'; -VIEWS: 'VIEWS'; +WARNINGS: 'WARNINGS'; WEEK: 'WEEK'; WHEN: 'WHEN'; WHERE: 'WHERE'; -WINDOW: 'WINDOW'; +WHITELIST: 'WHITELIST'; WITH: 'WITH'; -WITHIN: 'WITHIN'; +WORK: 'WORK'; +WORKLOAD: 'WORKLOAD'; +WRITE: 'WRITE'; YEAR: 'YEAR'; -ZONE: 'ZONE'; -DATEV2: 'DATEV2'; //--DORIS-KEYWORD-LIST-END //============================ // End of the keywords list @@ -417,13 +544,14 @@ GT : '>'; GTE : '>=' | '!<'; PLUS: '+'; -MINUS: '-'; +SUBTRACT: '-'; ASTERISK: '*'; SLASH: '/'; -PERCENT: '%'; +MOD: '%'; TILDE: '~'; AMPERSAND: '&'; LOGICALAND: '&&'; +LOGICALNOT: '!'; PIPE: '|'; DOUBLEPIPES: '||'; HAT: '^'; @@ -441,6 +569,11 @@ STRING_LITERAL | 'R"'(~'"')* '"' ; +LEADING_STRING + : LEFT_BRACE + | RIGHT_BRACE + ; + BIGINT_LITERAL : DIGIT+ 'L' ; diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index cd178cd22039f6b..a75e293dd5457e3 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -45,19 +45,22 @@ statement (WITH LABEL labelName=identifier)? cols=identifierList? // label and columns define (LEFT_BRACKET hints=identifierSeq RIGHT_BRACKET)? // hint define query #insertIntoQuery - | explain? UPDATE tableName=multipartIdentifier tableAlias + | explain? cte? UPDATE tableName=multipartIdentifier tableAlias SET updateAssignmentSeq fromClause? whereClause #update - | explain? DELETE FROM tableName=multipartIdentifier tableAlias + | explain? cte? DELETE FROM tableName=multipartIdentifier tableAlias (PARTITION partition=identifierList)? (USING relation (COMMA relation)*) whereClause #delete + | EXPORT TABLE tableName=multipartIdentifier + (PARTITION partition=identifierList)? + (whereClause)? + TO filePath=constant + (propertyClause)? + (withRemoteStorageSystem)? #export ; -propertiesStatment - : properties+=property (COMMA properties+=property)* - ; // -----------------Command accessories----------------- @@ -65,6 +68,7 @@ propertiesStatment identifierOrText : errorCapturingIdentifier | STRING_LITERAL + | LEADING_STRING ; userIdentify @@ -87,6 +91,22 @@ planType | ALL // default type ; +withRemoteStorageSystem + : WITH S3 LEFT_PAREN + brokerProperties=propertyItemList + RIGHT_PAREN + | WITH HDFS LEFT_PAREN + brokerProperties=propertyItemList + RIGHT_PAREN + | WITH LOCAL LEFT_PAREN + brokerProperties=propertyItemList + RIGHT_PAREN + | WITH BROKER brokerName=identifierOrText + (LEFT_PAREN + brokerProperties=propertyItemList + RIGHT_PAREN)? + ; + // -----------------Query----------------- // add queryOrganization for parse (q1) union (q2) union (q3) order by keys, otherwise 'order' will be recognized to be // identifier. @@ -94,18 +114,17 @@ planType outFileClause : INTO OUTFILE filePath=constant (FORMAT AS format=identifier)? - (PROPERTIES LEFT_PAREN properties+=property (COMMA properties+=property)* RIGHT_PAREN)? + (propertyClause)? ; query - : {!doris_legacy_SQL_syntax}? cte? queryTerm queryOrganization - | {doris_legacy_SQL_syntax}? queryTerm queryOrganization + : cte? queryTerm queryOrganization ; queryTerm - : queryPrimary #queryTermDefault + : queryPrimary #queryTermDefault | left=queryTerm operator=(UNION | EXCEPT | INTERSECT) - setQuantifier? right=queryTerm #setOperation + setQuantifier? right=queryTerm #setOperation ; setQuantifier @@ -114,19 +133,17 @@ setQuantifier ; queryPrimary - : querySpecification #queryPrimaryDefault - | TABLE multipartIdentifier #table - | LEFT_PAREN query RIGHT_PAREN #subquery + : querySpecification #queryPrimaryDefault + | LEFT_PAREN query RIGHT_PAREN #subquery ; querySpecification - : {doris_legacy_SQL_syntax}? cte? - selectClause + : selectClause fromClause? whereClause? aggClause? havingClause? - {doris_legacy_SQL_syntax}? queryOrganization #regularQuerySpecification + {doris_legacy_SQL_syntax}? queryOrganization #regularQuerySpecification ; cte @@ -199,7 +216,7 @@ havingClause selectHint: HINT_START hintStatements+=hintStatement (COMMA? hintStatements+=hintStatement)* HINT_END; hintStatement - : hintName=identifier LEFT_PAREN parameters+=hintAssignment (COMMA parameters+=hintAssignment)* RIGHT_PAREN + : hintName=identifier (LEFT_PAREN parameters+=hintAssignment (COMMA? parameters+=hintAssignment)* RIGHT_PAREN)? ; hintAssignment @@ -267,18 +284,29 @@ identifierSeq ; relationPrimary - : multipartIdentifier specifiedPartition? tabletList? tableAlias relationHint? lateralView* #tableName - | LEFT_PAREN query RIGHT_PAREN tableAlias lateralView* #aliasedQuery + : multipartIdentifier specifiedPartition? + tabletList? tableAlias relationHint? lateralView* #tableName + | LEFT_PAREN query RIGHT_PAREN tableAlias lateralView* #aliasedQuery | tvfName=identifier LEFT_PAREN - (properties+=property (COMMA properties+=property)*)? - RIGHT_PAREN tableAlias #tableValuedFunction + (properties=propertyItemList)? + RIGHT_PAREN tableAlias #tableValuedFunction + ; + +propertyClause + : PROPERTIES LEFT_PAREN fileProperties=propertyItemList RIGHT_PAREN ; -property - : key=propertyItem EQ value=propertyItem +propertyItemList + : properties+=propertyItem (COMMA properties+=propertyItem)* ; -propertyItem : identifier | constant ; +propertyItem + : key=propertyKey EQ value=propertyValue + ; + +propertyKey : identifier | constant ; + +propertyValue : identifier | constant ; tableAlias : (AS? strictIdentifier identifierList?)? @@ -306,7 +334,7 @@ expression ; booleanExpression - : NOT booleanExpression #logicalNot + : (LOGICALNOT | NOT) booleanExpression #logicalNot | EXISTS LEFT_PAREN query RIGHT_PAREN #exist | (ISNULL | IS_NULL_PRED) LEFT_PAREN valueExpression RIGHT_PAREN #isnull | IS_NOT_NULL_PRED LEFT_PAREN valueExpression RIGHT_PAREN #is_not_null_pred @@ -316,6 +344,8 @@ booleanExpression | left=booleanExpression operator=DOUBLEPIPES right=booleanExpression #doublePipes ; + + predicate : NOT? kind=BETWEEN lower=valueExpression AND upper=valueExpression | NOT? kind=(LIKE | REGEXP | RLIKE) pattern=valueExpression @@ -327,9 +357,9 @@ predicate valueExpression : primaryExpression #valueExpressionDefault - | operator=(MINUS | PLUS | TILDE) valueExpression #arithmeticUnary - | left=valueExpression operator=(ASTERISK | SLASH | PERCENT) right=valueExpression #arithmeticBinary - | left=valueExpression operator=(PLUS | MINUS | DIV | HAT | PIPE | AMPERSAND) + | operator=(SUBTRACT | PLUS | TILDE) valueExpression #arithmeticUnary + | left=valueExpression operator=(ASTERISK | SLASH | MOD) right=valueExpression #arithmeticBinary + | left=valueExpression operator=(PLUS | SUBTRACT | DIV | HAT | PIPE | AMPERSAND) right=valueExpression #arithmeticBinary | left=valueExpression comparisonOperator right=valueExpression #comparison | operator=(BITAND | BITOR | BITXOR) LEFT_PAREN left = valueExpression @@ -367,14 +397,37 @@ primaryExpression (INTERVAL unitsAmount=valueExpression unit=datetimeUnit | unitsAmount=valueExpression) RIGHT_PAREN #date_sub + | name=DATE_FLOOR + LEFT_PAREN + timestamp=valueExpression COMMA + (INTERVAL unitsAmount=valueExpression unit=datetimeUnit + | unitsAmount=valueExpression) + RIGHT_PAREN #dateFloor + | name=DATE_CEIL + LEFT_PAREN + timestamp=valueExpression COMMA + (INTERVAL unitsAmount=valueExpression unit=datetimeUnit + | unitsAmount=valueExpression) + RIGHT_PAREN #dateCeil | CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase | name=CAST LEFT_PAREN expression AS dataType RIGHT_PAREN #cast | constant #constantDefault + | interval #intervalLiteral | ASTERISK #star | qualifiedName DOT ASTERISK #star - | functionIdentifier LEFT_PAREN ((DISTINCT|ALL)? arguments+=expression - (COMMA arguments+=expression)* (ORDER BY sortItem (COMMA sortItem)*)?)? RIGHT_PAREN + | CHAR LEFT_PAREN + arguments+=expression (COMMA arguments+=expression)* + (USING charSet=identifierOrText)? + RIGHT_PAREN #charFunction + | CONVERT LEFT_PAREN argument=expression USING charSet=identifierOrText RIGHT_PAREN #convertCharSet + | CONVERT LEFT_PAREN argument=expression COMMA type=dataType RIGHT_PAREN #convertType + | functionIdentifier + LEFT_PAREN ( + (DISTINCT|ALL)? + arguments+=expression (COMMA arguments+=expression)* + (ORDER BY sortItem (COMMA sortItem)*)? + )? RIGHT_PAREN (OVER windowSpec)? #functionCall | value=primaryExpression LEFT_BRACKET index=valueExpression RIGHT_BRACKET #elementAt | value=primaryExpression LEFT_BRACKET begin=valueExpression @@ -385,8 +438,10 @@ primaryExpression | identifier #columnReference | base=primaryExpression DOT fieldName=identifier #dereference | LEFT_PAREN expression RIGHT_PAREN #parenthesizedExpression + | KEY (dbName=identifier DOT)? keyName=identifier #encryptKey | EXTRACT LEFT_PAREN field=identifier FROM (DATE | TIMESTAMP)? source=valueExpression RIGHT_PAREN #extract + | primaryExpression COLLATE (identifier | STRING_LITERAL | DEFAULT) #collate ; functionIdentifier @@ -395,7 +450,20 @@ functionIdentifier functionNameIdentifier : identifier - | LEFT | RIGHT + | ADD + | CONNECTION_ID + | CURRENT_CATALOG + | CURRENT_USER + | DATABASE + | IF + | LEFT + | LIKE + | PASSWORD + | REGEXP + | RIGHT + | SCHEMA + | TRIM + | USER ; windowSpec @@ -436,11 +504,14 @@ specifiedPartition constant : NULL #nullLiteral - | interval #intervalLiteral - | type=(DATE | DATEV2 | TIMESTAMP) STRING_LITERAL #typeConstructor + | type=(DATE | DATEV2 | TIMESTAMP) STRING_LITERAL #typeConstructor | number #numericLiteral | booleanValue #booleanLiteral - | STRING_LITERAL #stringLiteral + | STRING_LITERAL #stringLiteral + | LEFT_BRACKET items+=constant (COMMA items+=constant)* RIGHT_BRACKET #arrayLiteral + | LEFT_BRACE items+=constant COLON items+=constant + (COMMA items+=constant COLON items+=constant)* RIGHT_BRACE #mapLiteral + | LEFT_BRACE items+=constant (COMMA items+=constant)* RIGHT_BRACE #structLiteral ; comparisonOperator @@ -464,11 +535,40 @@ unitIdentifier ; dataType - : complex=ARRAY LT dataType GT #complexDataType - | complex=MAP LT dataType COMMA dataType GT #complexDataType - | complex=STRUCT LT complexColTypeList GT #complexDataType - | identifier (LEFT_PAREN INTEGER_VALUE - (COMMA INTEGER_VALUE)* RIGHT_PAREN)? #primitiveDataType + : complex=ARRAY LT dataType GT #complexDataType + | complex=MAP LT dataType COMMA dataType GT #complexDataType + | complex=STRUCT LT complexColTypeList GT #complexDataType + | primitiveColType (LEFT_PAREN (INTEGER_VALUE | ASTERISK) + (COMMA INTEGER_VALUE)* RIGHT_PAREN)? #primitiveDataType + ; + +primitiveColType: + | type=TINYINT + | type=SMALLINT + | (SIGNED | UNSIGNED)? type=INT + | type=BIGINT + | type=LARGEINT + | type=BOOLEAN + | type=FLOAT + | type=DOUBLE + | type=DATE + | type=DATETIME + | type=TIME + | type=DATEV2 + | type=DATETIMEV2 + | type=BITMAP + | type=QUANTILE_STATE + | type=HLL + | type=AGG_STATE + | type=STRING + | type=JSON + | type=JSONB + | type=TEXT + | type=VARCHAR + | type=CHAR + | type=DECIMAL + | type=DECIMALV3 + | type=ALL ; complexColTypeList @@ -492,7 +592,7 @@ errorCapturingIdentifier // extra left-factoring grammar errorCapturingIdentifierExtra - : (MINUS identifier)+ #errorIdent + : (SUBTRACT identifier)+ #errorIdent | #realIdent ; @@ -511,8 +611,8 @@ quotedIdentifier ; number - : MINUS? INTEGER_VALUE #integerLiteral - | MINUS? (EXPONENT_VALUE | DECIMAL_VALUE) #decimalLiteral + : SUBTRACT? INTEGER_VALUE #integerLiteral + | SUBTRACT? (EXPONENT_VALUE | DECIMAL_VALUE) #decimalLiteral ; // there are 1 kinds of keywords in Doris. @@ -522,275 +622,269 @@ number // TODO: need to stay consistent with the legacy nonReserved //--DEFAULT-NON-RESERVED-START - : ADD + : ADDDATE | AFTER - | ALL - | ALTER - | ANALYZE + | AGG_STATE + | AGGREGATE + | ALIAS | ANALYZED - | AND - | ANY - | ARCHIVE | ARRAY - | ASC | AT - | AUTHORIZATION - | AVG - | BETWEEN - | BOTH - | BUCKET + | AUTHORS + | BACKENDS + | BACKUP + | BEGIN + | BIN + | BITAND + | BITMAP + | BITMAP_UNION + | BITOR + | BITXOR + | BLOB + | BOOLEAN + | BRIEF + | BROKER | BUCKETS - | BY - | CACHE - | CASCADE - | CASE - | CAST + | BUILD + | BUILTIN + | CACHED | CATALOG | CATALOGS - | CHANGE + | CHAIN + | CHAR + | CHARSET | CHECK - | CLEAR | CLUSTER - | CLUSTERED - | CODEGEN - | COLLATE - | COLLECTION - | COLUMN + | CLUSTERS + | COLLATION | COLUMNS | COMMENT | COMMIT + | COMMITTED | COMPACT - | COMPACTIONS - | COMPUTE - | CONCATENATE - | CONSTRAINT - | COST - | CREATE - | CUBE - | CURRENT - | CURRENT_DATE - | CURRENT_TIME + | COMPLETE + | CONFIG + | CONNECTION + | CONNECTION_ID + | CONSISTENT + | CONVERT + | COPY + | COUNT + | CREATION + | CRON + | CURRENT_CATALOG | CURRENT_TIMESTAMP - | CURRENT_USER | DATA - | DATABASE - | DATABASES | DATE - | DATEV2 | DATE_ADD - | DATEDIFF + | DATE_CEIL | DATE_DIFF + | DATE_FLOOR + | DATE_SUB + | DATEADD + | DATEDIFF + | DATETIME + | DATETIMEV2 + | DATEV2 | DAY - | DBPROPERTIES - | DEFINED - | DELETE - | DELIMITED - | DESC - | DESCRIBE - | DFS - | DIRECTORIES - | DIRECTORY - | DISTINCT - | DISTRIBUTE - | DIV - | DROP - | ELSE + | DAYS_ADD + | DAYS_SUB + | DECIMAL + | DECIMALV3 + | DEFERRED + | DEMAND + | DIAGNOSE + | DISTINCTPC + | DISTINCTPCSA + | DO + | DYNAMIC + | ENABLE + | ENCRYPTKEY + | ENCRYPTKEYS | END - | ESCAPE - | ESCAPED - | EXCHANGE - | EXISTS - | EXPLAIN - | EXPORT - | EXTENDED + | ENDS + | ENGINE + | ENGINES + | ERRORS + | EVENTS + | EVERY + | EXCLUDE + | EXPIRED | EXTERNAL - | EXTRACT - | FALSE - | FETCH - | FILTER + | FAILED_LOGIN_ATTEMPTS + | FAST + | FEATURE | FIELDS - | FILEFORMAT + | FILE + | FILTER | FIRST - | FOLLOWING - | FOR - | FOREIGN | FORMAT - | FORMATTED - | FROM + | FREE + | FRONTENDS | FUNCTION - | FUNCTIONS | GLOBAL - | GRANT | GRAPH - | GROUP | GROUPING - | HAVING + | GROUPS + | HASH + | HDFS + | HELP + | HISTOGRAM + | HLL_UNION + | HOSTNAME | HOUR - | IF + | HUB + | IDENTIFIED | IGNORE - | IMPORT - | IN - | INDEX + | IMMEDIATE + | INCREMENTAL | INDEXES - | INPATH - | INPUTFORMAT - | INSERT - | INTERVAL - | INTO - | IS - | ITEMS - | KEYS + | INVERTED + | IS_NOT_NULL_PRED + | IS_NULL_PRED + | ISNULL + | ISOLATION + | JOB + | JOBS + | JSON + | JSONB | LABEL | LAST - | LAZY - | LEADING - | LIKE - | ILIKE - | LIMIT + | LDAP + | LDAP_ADMIN_PASSWORD + | LEFT_BRACE + | LESS + | LEVEL | LINES - | LIST - | LOAD + | LINK | LOCAL | LOCATION | LOCK - | LOCKS | LOGICAL - | MACRO | MAP - | MATCHED + | MATERIALIZED + | MAX + | MEMO | MERGE + | MIGRATE + | MIGRATIONS + | MIN | MINUTE + | MODIFY | MONTH - | MSCK - | NAMESPACE - | NAMESPACES + | MTMV + | NAME + | NAMES + | NEGATIVE + | NEVER + | NEXT + | NGRAM_BF | NO - | NOT - | NULL | NULLS | OF + | OFFSET | ONLY + | OPEN | OPTIMIZED - | OPTION - | OPTIONS - | OR - | ORDER - | OUT - | OUTER - | OUTPUTFORMAT - | OVERLAPS - | OVERLAY - | OVERWRITE + | PARAMETER | PARSED - | PARTITION - | PARTITIONED | PARTITIONS - | PERCENTILE_CONT - | PERCENTLIT + | PASSWORD + | PASSWORD_EXPIRE + | PASSWORD_HISTORY + | PASSWORD_LOCK_TIME + | PASSWORD_REUSE + | PATH + | PAUSE + | PERCENT + | PERIOD | PERMISSIVE | PHYSICAL - | PIVOT - | PLACING | PLAN + | PLUGIN + | PLUGINS | POLICY - | POSITION - | PRECEDING - | PRIMARY - | PRINCIPALS + | PROC + | PROCESSLIST + | PROFILE | PROPERTIES - | PURGE + | PROPERTY + | QUANTILE_STATE + | QUANTILE_UNION | QUERY - | RANGE - | RECORDREADER - | RECORDWRITER + | QUOTA + | RANDOM | RECOVER - | REDUCE - | REFERENCES + | RECYCLE | REFRESH - | RENAME - | REPAIR | REPEATABLE | REPLACE - | RESET - | RESPECT - | RESTRICT + | REPLACE_IF_NOT_NULL + | REPOSITORIES + | REPOSITORY + | RESOURCE + | RESOURCES + | RESTORE | RESTRICTIVE - | REVOKE + | RESUME + | RETURNS | REWRITTEN + | RIGHT_BRACE | RLIKE - | ROLE - | ROLES | ROLLBACK | ROLLUP - | ROWS + | ROUTINE + | S3 + | SAMPLE + | SCHEDULER | SCHEMA - | SCHEMAS | SECOND - | SELECT - | SEPARATED - | SERDE - | SERDEPROPERTIES - | SESSION_USER - | SET - | SETS - | SHOW - | SKEWED - | SOME - | SORT - | SORTED + | SERIALIZABLE + | SESSION + | SHAPE + | SKEW + | SNAPSHOT + | SONAME + | SPLIT | START - | STATISTICS + | STARTS + | STATS + | STATUS + | STOP | STORAGE - | STORED - | STRATIFY + | STREAM + | STREAMING + | STRING | STRUCT - | SYNC - | SYSTEM_TIME - | SYSTEM_VERSION - | TABLE + | SUBDATE + | SUM | TABLES - | TABLESAMPLE - | TBLPROPERTIES + | TASK + | TASKS | TEMPORARY - | TERMINATED - | THEN + | TEXT + | THAN | TIME | TIMESTAMP | TIMESTAMPADD | TIMESTAMPDIFF - | TO - | TOUCH - | TRAILING | TRANSACTION - | TRANSACTIONS - | TRANSFORM - | TRIM - | TRUE + | TRIGGERS | TRUNCATE - | TRY_CAST | TYPE - | UNARCHIVE - | UNBOUNDED - | UNCACHE - | UNIQUE - | UNKNOWN + | TYPES + | UNCOMMITTED | UNLOCK - | UNSET - | UPDATE - | USE | USER - | VALUES + | VALUE + | VARCHAR + | VARIABLES | VERBOSE | VERSION | VIEW - | VIEWS + | WARNINGS | WEEK - | WHEN - | WHERE - | WINDOW - | WITH - | WITHIN + | WORK | YEAR - | ZONE //--DEFAULT-NON-RESERVED-END - ; + ; \ No newline at end of file diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 19d4c220371e40f..6f37773c7ec7bbe 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -2680,7 +2680,7 @@ create_job_stmt ::= :} ; resume_job_stmt ::= - KW_RESUME KW_JOB job_label:jobLabel + KW_RESUME KW_JOB KW_FOR job_label:jobLabel {: RESULT = new ResumeJobStmt(jobLabel); :} @@ -6873,8 +6873,14 @@ timestamp_arithmetic_expr ::= // This function should not fully qualified throw new Exception("interval should not be qualified by database name"); } - - RESULT = new TimestampArithmeticExpr(functionName.getFunction(), l.get(0), v, u); + //eg: date_floor("0001-01-01 00:00:18",interval 5 second) convert to + //second_floor("0001-01-01 00:00:18", 5, "0001-01-01 00:00:00"); + if ("date_floor".equalsIgnoreCase(functionName.getFunction()) || + "date_ceil".equalsIgnoreCase(functionName.getFunction())) { + RESULT = FunctionCallExpr.functionWithIntervalConvert(functionName.getFunction().toLowerCase(), l.get(0), v, u); + } else { + RESULT = new TimestampArithmeticExpr(functionName.getFunction(), l.get(0), v, u); + } :} | function_name:functionName LPAREN time_unit:u COMMA expr:e1 COMMA expr:e2 RPAREN {: @@ -7221,6 +7227,10 @@ admin_stmt ::= {: RESULT = new AdminCleanTrashStmt(null); :} + | KW_ADMIN KW_SET KW_TABLE table_name:name KW_PARTITION KW_VERSION opt_properties:properties + {: + RESULT = new AdminSetPartitionVersionStmt(name, properties); + :} | KW_ADMIN KW_DIAGNOSE KW_TABLET INTEGER_LITERAL:tabletId {: RESULT = new AdminDiagnoseTabletStmt(tabletId); @@ -7237,6 +7247,10 @@ admin_stmt ::= {: RESULT = new AdminCopyTabletStmt(tabletId, properties); :} + | KW_ADMIN KW_SET KW_TABLE table_name:name KW_STATUS opt_properties:properties + {: + RESULT = new AdminSetTableStatusStmt(name, properties); + :} ; truncate_stmt ::= diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 596f76243c2db12..780317403742221 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -54,11 +54,9 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PartitionInfo; -import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.ReplicaAllocation; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf.TableType; -import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.View; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; @@ -189,20 +187,21 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, boolean needProcessOutsideTableLock = false; if (currentAlterOps.checkTableStoragePolicy(alterClauses)) { String tableStoragePolicy = olapTable.getStoragePolicy(); - if (!tableStoragePolicy.isEmpty()) { + String currentStoragePolicy = currentAlterOps.getTableStoragePolicy(alterClauses); + + // If the two policy has one same resource, then it's safe for the table to change policy + // There would only be the cooldown ttl or cooldown time would be affected + if (!Env.getCurrentEnv().getPolicyMgr() + .checkStoragePolicyIfSameResource(tableStoragePolicy, currentStoragePolicy) + && !tableStoragePolicy.isEmpty()) { for (Partition partition : olapTable.getAllPartitions()) { - for (Tablet tablet : partition.getBaseIndex().getTablets()) { - for (Replica replica : tablet.getReplicas()) { - if (replica.getRowCount() > 0 || replica.getDataSize() > 0) { - throw new DdlException("Do not support alter table's storage policy , this table [" - + olapTable.getName() + "] has storage policy " + tableStoragePolicy - + ", the table need to be empty."); - } - } + if (Partition.PARTITION_INIT_VERSION < partition.getVisibleVersion()) { + throw new DdlException("Do not support alter table's storage policy , this table [" + + olapTable.getName() + "] has storage policy " + tableStoragePolicy + + ", the table need to be empty."); } } } - String currentStoragePolicy = currentAlterOps.getTableStoragePolicy(alterClauses); // check currentStoragePolicy resource exist. Env.getCurrentEnv().getPolicyMgr().checkStoragePolicyExist(currentStoragePolicy); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetPartitionVersionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetPartitionVersionStmt.java new file mode 100644 index 000000000000000..a1edeb7a2c8ef8e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetPartitionVersionStmt.java @@ -0,0 +1,93 @@ +// 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.PropertyAnalyzer; +import org.apache.doris.common.util.Util; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; + +import java.util.Map; + +// Modify version of specified partition. Only used in emergency. +/* + * admin set table db.tbl partition version properties ("key" = "val", ..); + * "partition_id" = "20010", + * "visible_version" = "101" + */ +public class AdminSetPartitionVersionStmt extends DdlStmt { + private long partitionId = -1; + private long visibleVersion = -1; + private final TableName tableName; + private final Map properties; + + public AdminSetPartitionVersionStmt(TableName tableName, Map properties) { + this.tableName = tableName; + this.properties = properties; + } + + public String getDatabase() { + return tableName.getDb(); + } + + public String getTable() { + return tableName.getTbl(); + } + + public Long getPartitionId() { + return partitionId; + } + + public Long getVisibleVersion() { + return visibleVersion; + } + + @Override + public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + super.analyze(analyzer); + + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + tableName.analyze(analyzer); + Util.prohibitExternalCatalog(tableName.getCtl(), this.getClass().getSimpleName()); + + checkProperties(); + } + + private void checkProperties() throws AnalysisException { + partitionId = PropertyAnalyzer.analyzePartitionId(properties); + if (partitionId == -1) { + throw new AnalysisException("Should specify 'partition_id' property."); + } + visibleVersion = PropertyAnalyzer.analyzeVisibleVersion(properties); + if (visibleVersion == -1) { + throw new AnalysisException("Should specify 'visible_version' property."); + } + if (properties != null && !properties.isEmpty()) { + throw new AnalysisException("Unknown properties: " + properties.keySet()); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetTableStatusStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetTableStatusStmt.java new file mode 100644 index 000000000000000..e58d2af12785ea2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetTableStatusStmt.java @@ -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. + +package org.apache.doris.analysis; + +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable.OlapTableState; +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.Util; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; + +import lombok.Getter; + +import java.util.Map; + +public class AdminSetTableStatusStmt extends DdlStmt { + + public static final String TABLE_STATE = "state"; + + private final TableName tableName; + private final Map properties; + @Getter + private OlapTableState tableState; + + public AdminSetTableStatusStmt(TableName tableName, Map properties) { + this.tableName = tableName; + this.properties = properties; + } + + @Override + public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + super.analyze(analyzer); + + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + tableName.analyze(analyzer); + Util.prohibitExternalCatalog(tableName.getCtl(), this.getClass().getSimpleName()); + + checkProperties(); + } + + private void checkProperties() throws AnalysisException { + for (Map.Entry entry : properties.entrySet()) { + String key = entry.getKey(); + String val = entry.getValue(); + + if (key.equalsIgnoreCase(TABLE_STATE)) { + try { + tableState = OlapTableState.valueOf(val.toUpperCase()); + } catch (IllegalArgumentException e) { + throw new AnalysisException("Invalid table state: " + val); + } + } else { + throw new AnalysisException("Unsupported property: " + key); + } + } + + if (tableState == null) { + throw new AnalysisException("Should add properties: STATE."); + } + } + + public String getDbName() { + return tableName.getDb(); + } + + public String getTblName() { + return tableName.getTbl(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java index 5e49061bb7b8e3a..f0e96665096792a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java @@ -927,7 +927,34 @@ public SlotDescriptor registerColumnRef(TableName tblName, String colName) throw // =================================================== // Someone may concern that if t2 is not alias of t, this fix will cause incorrect resolve. In fact, // this does not happen, since we push t2.a in (1.2) down to this inline view, t2 must be alias of t. - if (d == null && isInlineView && newTblName.getTbl().equals(explicitViewAlias)) { + // create table tmp_can_drop_t1 ( + // cust_id varchar(96), + // user_id varchar(96) + // ) + // create table tmp_can_drop_t2 ( + // cust_id varchar(96), + // usr_id varchar(96) + // ) + // select + // a.cust_id, + // a.usr_id + // from ( + // select + // a.cust_id, + // a.usr_id, --------->(report error, because there is no user_id column in tmp_can_drop_t1) + // a.user_id + // from tmp_can_drop_t1 a + // full join ( + // select + // cust_id, + // usr_id + // from + // tmp_can_drop_t2 + // ) b + // on b.cust_id = a.cust_id + // ) a; + if (d == null && isInlineView && newTblName.getTbl().equals(explicitViewAlias) + && !tupleByAlias.containsKey(newTblName.getTbl())) { d = resolveColumnRef(colName); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java index a1f9b6bd82c40b0..999d630153db7e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java @@ -85,7 +85,8 @@ public class CreateJobStmt extends DdlStmt { private String timezone = TimeUtils.DEFAULT_TIME_ZONE; private static final ImmutableSet> supportStmtSuperClass - = new ImmutableSet.Builder>().add(InsertStmt.class).build(); + = new ImmutableSet.Builder>().add(InsertStmt.class) + .add(UpdateStmt.class).build(); private static HashSet supportStmtClassNamesCache = new HashSet<>(16); 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 11d17ec816255ad..8a2f693a553f0f9 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 @@ -1046,19 +1046,19 @@ public LocalDateTime getTimeFormatter() { return LocalDateTime.of(year, month, dayOfMonth, hour, minute, second, microSeconds * 1000); } - public DateLiteral plusYears(int year) throws AnalysisException { + public DateLiteral plusYears(long year) throws AnalysisException { return new DateLiteral(getTimeFormatter().plusYears(year), type); } - public DateLiteral plusMonths(int month) throws AnalysisException { + public DateLiteral plusMonths(long month) throws AnalysisException { return new DateLiteral(getTimeFormatter().plusMonths(month), type); } - public DateLiteral plusDays(int day) throws AnalysisException { + public DateLiteral plusDays(long day) throws AnalysisException { return new DateLiteral(getTimeFormatter().plusDays(day), type); } - public DateLiteral plusHours(int hour) throws AnalysisException { + public DateLiteral plusHours(long hour) throws AnalysisException { if (type.isDate()) { return new DateLiteral(getTimeFormatter().plusHours(hour), Type.DATETIME); } @@ -1068,7 +1068,7 @@ public DateLiteral plusHours(int hour) throws AnalysisException { return new DateLiteral(getTimeFormatter().plusHours(hour), type); } - public DateLiteral plusMinutes(int minute) { + public DateLiteral plusMinutes(long minute) { if (type.isDate()) { return new DateLiteral(getTimeFormatter().plusMinutes(minute), Type.DATETIME); } @@ -1078,7 +1078,7 @@ public DateLiteral plusMinutes(int minute) { return new DateLiteral(getTimeFormatter().plusMinutes(minute), type); } - public DateLiteral plusSeconds(int second) { + public DateLiteral plusSeconds(long second) { if (type.isDate()) { return new DateLiteral(getTimeFormatter().plusSeconds(second), Type.DATETIME); } @@ -1536,6 +1536,10 @@ private long strToLong(String l) throws InvalidFormatException { } } + public long daynr() { + return calcDaynr(this.year, this.month, this.day); + } + // calculate the number of days from year 0000-00-00 to year-month-day private long calcDaynr(long year, long month, long day) { long delsum = 0; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java index 69f4d7174f46ca5..cc0d0883e6c44d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java @@ -24,6 +24,7 @@ import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; +import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.FeNameFormat; @@ -50,6 +51,7 @@ import java.util.Map; import java.util.Optional; import java.util.UUID; +import java.util.stream.Collectors; // EXPORT statement, export data to dirs by broker. // @@ -65,7 +67,6 @@ public class ExportStmt extends StatementBase { private static final String DEFAULT_COLUMN_SEPARATOR = "\t"; private static final String DEFAULT_LINE_DELIMITER = "\n"; - private static final String DEFAULT_COLUMNS = ""; private static final String DEFAULT_PARALLELISM = "1"; private static final ImmutableSet PROPERTIES_SET = new ImmutableSet.Builder() @@ -85,6 +86,7 @@ public class ExportStmt extends StatementBase { private TableName tblName; private List partitionStringNames; private Expr whereExpr; + private String whereSql; private String path; private BrokerDesc brokerDesc; private Map properties = Maps.newHashMap(); @@ -121,13 +123,21 @@ public ExportStmt(TableRef tableRef, Expr whereExpr, String path, this.brokerDesc = brokerDesc; this.columnSeparator = DEFAULT_COLUMN_SEPARATOR; this.lineDelimiter = DEFAULT_LINE_DELIMITER; - this.columns = DEFAULT_COLUMNS; Optional optionalSessionVariable = Optional.ofNullable( ConnectContext.get().getSessionVariable()); this.sessionVariables = optionalSessionVariable.orElse(VariableMgr.getDefaultSessionVariable()); } + /** + * This constructor used by nereids planner + */ + public ExportStmt(TableRef tableRef, String whereSql, String path, + Map properties, BrokerDesc brokerDesc) { + this(tableRef, (Expr) null, path, properties, brokerDesc); + this.whereSql = whereSql; + } + @Override public boolean needAuditEncryption() { return brokerDesc != null; @@ -209,6 +219,7 @@ private void setJob() throws UserException { // set where expr exportJob.setWhereExpr(this.whereExpr); + exportJob.setWhereSql(this.whereSql); // set path exportJob.setExportPath(this.path); @@ -223,7 +234,7 @@ private void setJob() throws UserException { exportJob.setMaxFileSize(this.maxFileSize); exportJob.setDeleteExistingFiles(this.deleteExistingFiles); - if (!Strings.isNullOrEmpty(this.columns)) { + if (columns != null) { Splitter split = Splitter.on(',').trimResults().omitEmptyStrings(); exportJob.setExportColumns(split.splitToList(this.columns.toLowerCase())); } @@ -343,7 +354,14 @@ private void checkProperties(Map properties) throws UserExceptio properties, ExportStmt.DEFAULT_COLUMN_SEPARATOR)); this.lineDelimiter = Separator.convertSeparator(PropertyAnalyzer.analyzeLineDelimiter( properties, ExportStmt.DEFAULT_LINE_DELIMITER)); - this.columns = properties.getOrDefault(LoadStmt.KEY_IN_PARAM_COLUMNS, DEFAULT_COLUMNS); + // null means not specified + // "" means user specified zero columns + this.columns = properties.getOrDefault(LoadStmt.KEY_IN_PARAM_COLUMNS, null); + + // check columns are exits + if (columns != null) { + checkColumns(); + } // format this.format = properties.getOrDefault(LoadStmt.KEY_IN_PARAM_FORMAT_TYPE, "csv").toLowerCase(); @@ -370,6 +388,24 @@ private void checkProperties(Map properties) throws UserExceptio } } + private void checkColumns() throws DdlException { + if (this.columns.isEmpty()) { + throw new DdlException("columns can not be empty"); + } + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(this.tblName.getDb()); + Table table = db.getTableOrDdlException(this.tblName.getTbl()); + List tableColumns = table.getBaseSchema().stream().map(column -> column.getName()) + .collect(Collectors.toList()); + Splitter split = Splitter.on(',').trimResults().omitEmptyStrings(); + + List columnsSpecified = split.splitToList(this.columns.toLowerCase()); + for (String columnName : columnsSpecified) { + if (!tableColumns.contains(columnName)) { + throw new DdlException("unknown column [" + columnName + "] in table [" + this.tblName.getTbl() + "]"); + } + } + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java index 9abd91acad7c261..c9af3ed8740e97c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java @@ -478,7 +478,30 @@ protected Expr substituteImpl(ExprSubstitutionMap smap, ExprSubstitutionMap disj aggFnParams = aggFnParams .clone(newParams); } - return super.substituteImpl(smap, disjunctsMap, analyzer); + if (isImplicitCast()) { + return getChild(0).substituteImpl(smap, disjunctsMap, analyzer); + } + if (smap != null) { + Expr substExpr = smap.get(this); + if (substExpr != null) { + return substExpr.clone(); + } + } + if (Expr.IS_OR_PREDICATE.apply(this) && disjunctsMap != null) { + smap = disjunctsMap; + disjunctsMap = null; + } + for (int i = 0; i < children.size(); ++i) { + // we shouldn't change literal expr in function call expr + if (!(children.get(i) instanceof LiteralExpr)) { + children.set(i, children.get(i).substituteImpl(smap, disjunctsMap, analyzer)); + } + } + // SlotRefs must remain analyzed to support substitution across query blocks. All + // other exprs must be analyzed again after the substitution to add implicit casts + // and for resolving their correct function signature. + resetAnalysisState(); + return this; } @Override @@ -534,8 +557,8 @@ private String paramsToSql() { || fnName.getFunction().equalsIgnoreCase("hours_diff") || fnName.getFunction().equalsIgnoreCase("minutes_diff") || fnName.getFunction().equalsIgnoreCase("seconds_diff")) { - sb.append(children.get(1).toSql()).append(", "); - sb.append(children.get(0).toSql()).append(")"); + sb.append(children.get(0).toSql()).append(", "); + sb.append(children.get(1).toSql()).append(")"); return sb.toString(); } // used by nereids END @@ -1157,7 +1180,8 @@ private void analyzeArrayFunction(Analyzer analyzer) throws AnalysisException { || fnName.getFunction().equalsIgnoreCase("array_cum_sum") || fnName.getFunction().equalsIgnoreCase("array_intersect") || fnName.getFunction().equalsIgnoreCase("arrays_overlap") - || fnName.getFunction().equalsIgnoreCase("array_concat")) { + || fnName.getFunction().equalsIgnoreCase("array_concat") + || fnName.getFunction().equalsIgnoreCase("array")) { Type[] childTypes = collectChildReturnTypes(); Type compatibleType = childTypes[0]; for (int i = 1; i < childTypes.length; ++i) { @@ -1173,9 +1197,7 @@ private void analyzeArrayFunction(Analyzer analyzer) throws AnalysisException { for (int i = 0; i < childTypes.length; i++) { uncheckedCastChild(compatibleType, i); } - } - - if (fnName.getFunction().equalsIgnoreCase("array_exists")) { + } else if (fnName.getFunction().equalsIgnoreCase("array_exists")) { Type[] newArgTypes = new Type[1]; if (!(getChild(0) instanceof CastExpr)) { Expr castExpr = getChild(0).castTo(ArrayType.create(Type.BOOLEAN, true)); @@ -1190,13 +1212,14 @@ private void analyzeArrayFunction(Analyzer analyzer) throws AnalysisException { throw new AnalysisException(getFunctionNotFoundError(collectChildReturnTypes())); } fn.setReturnType(getChild(0).getType()); - } - - // make nested type with function param can be Compatible otherwise be will not deal with type - if (fnName.getFunction().equalsIgnoreCase("array_position") + } else if (fnName.getFunction().equalsIgnoreCase("array_position") || fnName.getFunction().equalsIgnoreCase("array_contains") || fnName.getFunction().equalsIgnoreCase("countequal")) { + // make nested type with function param can be Compatible otherwise be will not deal with type Type[] childTypes = collectChildReturnTypes(); + if (childTypes[0].isNull()) { + childTypes[0] = new ArrayType(Type.NULL); + } Type compatibleType = ((ArrayType) childTypes[0]).getItemType(); for (int i = 1; i < childTypes.length; ++i) { compatibleType = Type.getAssignmentCompatibleType(compatibleType, childTypes[i], true); @@ -1694,8 +1717,10 @@ && collectChildReturnTypes()[0].isDecimalV3()) { || (children.get(0).getType().isDecimalV2() && ((ArrayType) args[ix]).getItemType().isDecimalV2()))) { continue; - } else if ((fnName.getFunction().equalsIgnoreCase("array_distinct") || fnName.getFunction() - .equalsIgnoreCase("array_remove") || fnName.getFunction().equalsIgnoreCase("array_sort") + } else if ((fnName.getFunction().equalsIgnoreCase("array") + || fnName.getFunction().equalsIgnoreCase("array_distinct") + || fnName.getFunction().equalsIgnoreCase("array_remove") + || fnName.getFunction().equalsIgnoreCase("array_sort") || fnName.getFunction().equalsIgnoreCase("array_reverse_sort") || fnName.getFunction().equalsIgnoreCase("array_overlap") || fnName.getFunction().equalsIgnoreCase("array_union") @@ -2242,4 +2267,17 @@ public Function findUdf(FunctionName fnName, Analyzer analyzer) throws AnalysisE } return fn; } + + // eg: date_floor("0001-01-01 00:00:18",interval 5 second) convert to + // second_floor("0001-01-01 00:00:18", 5, "0001-01-01 00:00:00"); + public static FunctionCallExpr functionWithIntervalConvert(String functionName, Expr str, Expr interval, + String timeUnitIdent) throws AnalysisException { + String newFunctionName = timeUnitIdent + "_" + functionName.split("_")[1]; + List params = new ArrayList<>(); + Expr defaultDatetime = new DateLiteral(0001, 01, 01, 0, 0, 0, 0, Type.DATETIMEV2); + params.add(str); + params.add(interval); + params.add(defaultDatetime); + return new FunctionCallExpr(newFunctionName, params); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java index 2e2eed316b25862..bde041aeefa73e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java @@ -402,10 +402,4 @@ public static int getParmLen(ByteBuffer data) { public boolean matchExprs(List exprs, SelectStmt stmt, boolean ignoreAlias, TupleDescriptor tuple) { return true; } - - @Override - protected Expr substituteImpl(ExprSubstitutionMap smap, ExprSubstitutionMap disjunctsMap, - Analyzer analyzer) { - return this; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java index d4401626cfa0a58..eb19620fc82deb2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java @@ -121,6 +121,7 @@ public class LoadStmt extends DdlStmt { public static final String KEY_IN_PARAM_BACKEND_ID = "backend_id"; public static final String KEY_SKIP_LINES = "skip_lines"; public static final String KEY_TRIM_DOUBLE_QUOTES = "trim_double_quotes"; + public static final String PARTIAL_COLUMNS = "partial_columns"; public static final String KEY_COMMENT = "comment"; @@ -162,6 +163,12 @@ public class LoadStmt extends DdlStmt { return Boolean.valueOf(s); } }) + .put(PARTIAL_COLUMNS, new Function() { + @Override + public @Nullable Boolean apply(@Nullable String s) { + return Boolean.valueOf(s); + } + }) .put(TIMEZONE, new Function() { @Override public @Nullable String apply(@Nullable String s) { @@ -346,6 +353,15 @@ public static void checkProperties(Map properties) throws DdlExc } } + // partial update + final String partialColumnsProperty = properties.get(PARTIAL_COLUMNS); + if (partialColumnsProperty != null) { + if (!partialColumnsProperty.equalsIgnoreCase("true") + && !partialColumnsProperty.equalsIgnoreCase("false")) { + throw new DdlException(PARTIAL_COLUMNS + " is not a boolean"); + } + } + // time zone final String timezone = properties.get(TIMEZONE); if (timezone != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/MVColumnItem.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/MVColumnItem.java index 01e0b01b403a155..d61a3ea862f69fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/MVColumnItem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/MVColumnItem.java @@ -92,6 +92,7 @@ public MVColumnItem(Expr defineExpr) throws AnalysisException { this.type = defineExpr.getType(); if (this.type instanceof ScalarType && this.type.isStringType()) { + this.type = new ScalarType(type.getPrimitiveType()); ((ScalarType) this.type).setMaxLength(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/MapLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/MapLiteral.java index 1130868e662c4ba..ba9878977c3026c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/MapLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/MapLiteral.java @@ -26,6 +26,7 @@ import org.apache.doris.thrift.TTypeNode; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import org.apache.commons.lang3.StringUtils; import java.io.DataInput; @@ -44,6 +45,15 @@ public MapLiteral() { children = new ArrayList<>(); } + public MapLiteral(Type type, List keys, List values) { + this.type = type; + children = Lists.newArrayList(); + for (int i = 0; i < keys.size(); i++) { + children.add(keys.get(i)); + children.add(values.get(i)); + } + } + public MapLiteral(LiteralExpr... exprs) throws AnalysisException { Type keyType = Type.NULL; Type valueType = Type.NULL; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java index f1e6031d23dd0b9..10c958c47b0bb6a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java @@ -68,8 +68,9 @@ public enum SchemaTableType { SCH_CREATE_TABLE("CREATE_TABLE", "CREATE_TABLE", TSchemaTableType.SCH_CREATE_TABLE), SCH_INVALID("NULL", "NULL", TSchemaTableType.SCH_INVALID), SCH_ROWSETS("ROWSETS", "ROWSETS", TSchemaTableType.SCH_ROWSETS), - SCH_PARAMETERS("PARAMETERS", "PARAMETERS", TSchemaTableType.SCH_PARAMETERS); - + SCH_PARAMETERS("PARAMETERS", "PARAMETERS", TSchemaTableType.SCH_PARAMETERS), + SCH_METADATA_NAME_IDS("METADATA_NAME_IDS", "METADATA_NAME_IDS", TSchemaTableType.SCH_METADATA_NAME_IDS), + SCH_PROFILING("PROFILING", "PROFILING", TSchemaTableType.SCH_PROFILING); private static final String dbName = "INFORMATION_SCHEMA"; private static SelectList fullSelectLists; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java index 1eb0241cfe20ea3..e7b04d9eceda620 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java @@ -43,6 +43,7 @@ public class ShowJobTaskStmt extends ShowStmt { .add("StartTime") .add("EndTime") .add("Status") + .add("Result") .add("ErrorMsg") .build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogGcer.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogGcer.java index 6dbe47ea220ac53..468e5a08186422e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogGcer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogGcer.java @@ -58,7 +58,7 @@ protected void runAfterCatalogReady() { if (tombstones != null && !tombstones.isEmpty()) { LOG.info("tomebstones size: {}", tombstones.size()); } else { - LOG.info("no gc binlogg"); + LOG.info("no gc binlog"); return; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index 12d56ab7331ad4b..3dd3044404e0847 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -82,6 +82,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Cardinality; import org.apache.doris.nereids.trees.expressions.functions.scalar.Cbrt; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ceil; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Char; import org.apache.doris.nereids.trees.expressions.functions.scalar.CharacterLength; import org.apache.doris.nereids.trees.expressions.functions.scalar.Coalesce; import org.apache.doris.nereids.trees.expressions.functions.scalar.Concat; @@ -92,6 +93,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ConvertTz; import org.apache.doris.nereids.trees.expressions.functions.scalar.Cos; import org.apache.doris.nereids.trees.expressions.functions.scalar.CountEqual; +import org.apache.doris.nereids.trees.expressions.functions.scalar.CreateMap; +import org.apache.doris.nereids.trees.expressions.functions.scalar.CreateNamedStruct; +import org.apache.doris.nereids.trees.expressions.functions.scalar.CreateStruct; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentCatalog; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentDate; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentTime; @@ -196,6 +200,11 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Lpad; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ltrim; import org.apache.doris.nereids.trees.expressions.functions.scalar.MakeDate; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapContainsKey; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapContainsValue; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapKeys; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapSize; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapValues; import org.apache.doris.nereids.trees.expressions.functions.scalar.Mask; import org.apache.doris.nereids.trees.expressions.functions.scalar.MaskFirstN; import org.apache.doris.nereids.trees.expressions.functions.scalar.MaskLastN; @@ -299,6 +308,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.StrLeft; import org.apache.doris.nereids.trees.expressions.functions.scalar.StrRight; import org.apache.doris.nereids.trees.expressions.functions.scalar.StrToDate; +import org.apache.doris.nereids.trees.expressions.functions.scalar.StructElement; import org.apache.doris.nereids.trees.expressions.functions.scalar.SubBitmap; import org.apache.doris.nereids.trees.expressions.functions.scalar.SubReplace; import org.apache.doris.nereids.trees.expressions.functions.scalar.Substring; @@ -419,6 +429,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Cardinality.class, "array_size", "cardinality", "size"), scalar(Cbrt.class, "cbrt"), scalar(Ceil.class, "ceil", "ceiling"), + scalar(Char.class, "char"), scalar(CharacterLength.class, "char_length", "character_length"), scalar(Coalesce.class, "coalesce"), scalar(Concat.class, "concat"), @@ -429,6 +440,9 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(ConvertTz.class, "convert_tz"), scalar(Cos.class, "cos"), scalar(CountEqual.class, "countequal"), + scalar(CreateMap.class, "map"), + scalar(CreateStruct.class, "struct"), + scalar(CreateNamedStruct.class, "named_struct"), scalar(CurrentCatalog.class, "current_catalog"), scalar(CurrentDate.class, "curdate", "current_date"), scalar(CurrentTime.class, "curtime", "current_time"), @@ -533,6 +547,11 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Lpad.class, "lpad"), scalar(Ltrim.class, "ltrim"), scalar(MakeDate.class, "makedate"), + scalar(MapContainsKey.class, "map_contains_key"), + scalar(MapContainsValue.class, "map_contains_value"), + scalar(MapKeys.class, "map_keys"), + scalar(MapSize.class, "map_size"), + scalar(MapValues.class, "map_values"), scalar(Mask.class, "mask"), scalar(MaskFirstN.class, "mask_first_n"), scalar(MaskLastN.class, "mask_last_n"), @@ -598,6 +617,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Sign.class, "sign"), scalar(Sin.class, "sin"), scalar(Sleep.class, "sleep"), + scalar(StructElement.class, "struct_element"), scalar(Sm3.class, "sm3"), scalar(Sm3sum.class, "sm3sum"), scalar(Sm4Decrypt.class, "sm4_decrypt"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DataProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DataProperty.java index 7028e5e449ad9ce..731776384d046a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DataProperty.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DataProperty.java @@ -98,6 +98,10 @@ public String getStoragePolicy() { return storagePolicy; } + public void setStoragePolicy(String storagePolicy) { + this.storagePolicy = storagePolicy; + } + public boolean isStorageMediumSpecified() { return storageMediumSpecified; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index 8da88fa146c87b4..82f6f3966266bbf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -611,6 +611,12 @@ private void discardHudiTable() { } } + public void analyze() { + for (Table table : nameToTable.values()) { + table.analyze(getFullName()); + } + } + @Override public void readFields(DataInput in) throws IOException { super.readFields(in); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java index f9c5d94601f787e..8c2833764123fa1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java @@ -206,6 +206,15 @@ default T getTableOrDdlException(long tableId) throws DdlException { return getTableOrException(tableId, t -> new DdlException(ErrorCode.ERR_BAD_TABLE_ERROR.formatErrorMsg(t))); } + default T getTableOrDdlException(long tableId, TableIf.TableType tableType) throws DdlException { + T table = getTableOrDdlException(tableId); + if (table.getType() != tableType) { + throw new DdlException( + "table type is not " + tableType + ", tableId=" + tableId + ", type=" + table.getType()); + } + return table; + } + default T getTableOrAnalysisException(String tableName) throws AnalysisException { return getTableOrException(tableName, t -> new AnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR.formatErrorMsg(t))); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index db8ac8263eb9ec4..af86bb32ef5375d 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -30,7 +30,9 @@ import org.apache.doris.analysis.AdminCleanTrashStmt; import org.apache.doris.analysis.AdminCompactTableStmt; import org.apache.doris.analysis.AdminSetConfigStmt; +import org.apache.doris.analysis.AdminSetPartitionVersionStmt; import org.apache.doris.analysis.AdminSetReplicaStatusStmt; +import org.apache.doris.analysis.AdminSetTableStatusStmt; import org.apache.doris.analysis.AlterDatabasePropertyStmt; import org.apache.doris.analysis.AlterDatabaseQuotaStmt; import org.apache.doris.analysis.AlterDatabaseQuotaStmt.QuotaType; @@ -85,6 +87,7 @@ import org.apache.doris.catalog.DistributionInfo.DistributionInfoType; import org.apache.doris.catalog.MaterializedIndex.IndexExtState; import org.apache.doris.catalog.MetaIdGenerator.IdGeneratorBuffer; +import org.apache.doris.catalog.OlapTable.OlapTableState; import org.apache.doris.catalog.Replica.ReplicaStatus; import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.clone.ColocateTableCheckerAndBalancer; @@ -190,7 +193,9 @@ import org.apache.doris.persist.RefreshExternalTableInfo; import org.apache.doris.persist.ReplacePartitionOperationLog; import org.apache.doris.persist.ReplicaPersistInfo; +import org.apache.doris.persist.SetPartitionVersionOperationLog; import org.apache.doris.persist.SetReplicaStatusOperationLog; +import org.apache.doris.persist.SetTableStatusOperationLog; import org.apache.doris.persist.Storage; import org.apache.doris.persist.StorageInfo; import org.apache.doris.persist.TableInfo; @@ -236,6 +241,7 @@ import org.apache.doris.task.PriorityMasterTaskExecutor; import org.apache.doris.thrift.BackendService; import org.apache.doris.thrift.TCompressionType; +import org.apache.doris.thrift.TFrontendInfo; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TStorageMedium; import org.apache.doris.transaction.DbUsedDataQuotaInfoCollector; @@ -472,6 +478,19 @@ public class Env { private HiveTransactionMgr hiveTransactionMgr; + public List getFrontendInfos() { + List res = new ArrayList<>(); + + for (Frontend fe : frontends.values()) { + TFrontendInfo feInfo = new TFrontendInfo(); + feInfo.setCoordinatorAddress(new TNetworkAddress(fe.getHost(), fe.getRpcPort())); + feInfo.setProcessUuid(fe.getProcessUUID()); + res.add(feInfo); + } + + return res; + } + public List getFrontends(FrontendNodeType nodeType) { if (nodeType == null) { // get all @@ -994,7 +1013,7 @@ private void getClusterIdAndRole() throws IOException { // For compatibility. Because this is the very first time to start, so we arbitrarily choose // a new name for this node role = FrontendNodeType.FOLLOWER; - nodeName = genFeNodeName(selfNode.getIdent(), + nodeName = genFeNodeName(selfNode.getHost(), selfNode.getPort(), false /* new style */); storage.writeFrontendRoleAndNodeName(role, nodeName); LOG.info("very first time to start this node. role: {}, node name: {}", role.name(), nodeName); @@ -5251,6 +5270,40 @@ public void checkTablets(AdminCheckTabletsStmt stmt) { } } + public void setTableStatus(AdminSetTableStatusStmt stmt) throws MetaNotFoundException { + String dbName = stmt.getDbName(); + String tableName = stmt.getTblName(); + setTableStatusInternal(dbName, tableName, stmt.getTableState(), false); + } + + public void replaySetTableStatus(SetTableStatusOperationLog log) throws MetaNotFoundException { + setTableStatusInternal(log.getDbName(), log.getTblName(), log.getState(), true); + } + + public void setTableStatusInternal(String dbName, String tableName, OlapTableState state, boolean isReplay) + throws MetaNotFoundException { + Database db = getInternalCatalog().getDbOrMetaException(dbName); + OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP); + olapTable.writeLockOrMetaException(); + try { + OlapTableState oldState = olapTable.getState(); + if (state != null && oldState != state) { + olapTable.setState(state); + if (!isReplay) { + SetTableStatusOperationLog log = new SetTableStatusOperationLog(dbName, tableName, state); + editLog.logSetTableStatus(log); + } + LOG.info("set table {} state from {} to {}. is replay: {}.", + tableName, oldState, state, isReplay); + } else { + LOG.warn("ignore set same state {} for table {}. is replay: {}.", + olapTable.getState(), tableName, isReplay); + } + } finally { + olapTable.writeUnlock(); + } + } + // Set specified replica's status. If replica does not exist, just ignore it. public void setReplicaStatus(AdminSetReplicaStatusStmt stmt) throws MetaNotFoundException { long tabletId = stmt.getTabletId(); @@ -5386,6 +5439,55 @@ public void cleanTrash(AdminCleanTrashStmt stmt) { } } + public void setPartitionVersion(AdminSetPartitionVersionStmt stmt) throws DdlException { + String database = stmt.getDatabase(); + String table = stmt.getTable(); + long partitionId = stmt.getPartitionId(); + long visibleVersion = stmt.getVisibleVersion(); + int setSuccess = setPartitionVersionInternal(database, table, partitionId, visibleVersion, false); + if (setSuccess == -1) { + throw new DdlException("Failed to set partition visible version to " + visibleVersion + ". " + "Partition " + + partitionId + " not exists. Database " + database + ", Table " + table + "."); + } + } + + public void replaySetPartitionVersion(SetPartitionVersionOperationLog log) throws DdlException { + int setSuccess = setPartitionVersionInternal(log.getDatabase(), log.getTable(), + log.getPartitionId(), log.getVisibleVersion(), true); + if (setSuccess == -1) { + LOG.warn("Failed to set partition visible version to {}. " + + "Database {}, Table {}, Partition {} not exists.", log.getDatabase(), log.getTable(), + log.getVisibleVersion(), log.getPartitionId()); + } + } + + public int setPartitionVersionInternal(String database, String table, long partitionId, + long visibleVersion, boolean isReplay) throws DdlException { + int result = -1; + Database db = getInternalCatalog().getDbOrDdlException(database); + OlapTable olapTable = db.getOlapTableOrDdlException(table); + olapTable.writeLockOrDdlException(); + try { + Partition partition = olapTable.getPartition(partitionId); + if (partition != null) { + Long oldVersion = partition.getVisibleVersion(); + partition.updateVisibleVersion(visibleVersion); + partition.setNextVersion(visibleVersion + 1); + result = 0; + if (!isReplay) { + SetPartitionVersionOperationLog log = new SetPartitionVersionOperationLog( + database, table, partitionId, visibleVersion); + getEditLog().logSetPartitionVersion(log); + } + LOG.info("set partition {} visible version from {} to {}. Database {}, Table {}, is replay:" + + " {}.", partitionId, oldVersion, visibleVersion, database, table, isReplay); + } + } finally { + olapTable.writeUnlock(); + } + return result; + } + public static boolean isStoredTableNamesLowerCase() { return GlobalVariable.lowerCaseTableNames == 1; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java index dcf144b61554c4f..e9fc898b2e1d691 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java @@ -300,11 +300,9 @@ public static String handleJdbcUrl(String jdbcUrl) throws DdlException { newJdbcUrl = checkAndSetJdbcBoolParam(newJdbcUrl, "yearIsDateType", "true", "false"); // MySQL Types and Return Values for GetColumnTypeName and GetColumnClassName // are presented in https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-reference-type-conversions.html - // However when tinyInt1isBit=false, GetColumnClassName of MySQL returns java.lang.Boolean, - // while that of Doris returns java.lang.Integer. In order to be compatible with both MySQL and Doris, - // Jdbc params should set tinyInt1isBit=true&transformedBitIsBoolean=true + // When mysql's tinyint stores non-0 or 1, we need to read the data correctly, + // so we need tinyInt1isBit=false newJdbcUrl = checkAndSetJdbcBoolParam(newJdbcUrl, "tinyInt1isBit", "true", "false"); - newJdbcUrl = checkAndSetJdbcBoolParam(newJdbcUrl, "transformedBitIsBoolean", "false", "true"); // set useUnicode and characterEncoding to false and utf-8 newJdbcUrl = checkAndSetJdbcBoolParam(newJdbcUrl, "useUnicode", "false", "true"); newJdbcUrl = checkAndSetJdbcBoolParam(newJdbcUrl, "rewriteBatchedStatements", "false", "true"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java index eb5e7b64a955b41..57d428d481fd957 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java @@ -28,10 +28,8 @@ import org.apache.doris.common.util.SqlParserUtils; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; -import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; import org.apache.doris.qe.SqlModeHelper; -import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TStorageType; import com.google.common.base.Preconditions; @@ -292,16 +290,9 @@ public static MaterializedIndexMeta read(DataInput in) throws IOException { @Override public void gsonPostProcess() throws IOException { initColumnNameMap(); - parseStmt(null); } public void parseStmt(Analyzer analyzer) throws IOException { - if (analyzer == null && dbName != null) { - ConnectContext connectContext = new ConnectContext(); - connectContext.setCluster(SystemInfoService.DEFAULT_CLUSTER); - connectContext.setDatabase(dbName); - analyzer = new Analyzer(Env.getCurrentEnv(), connectContext); - } // analyze define stmt if (defineStmt == null) { return; @@ -316,7 +307,7 @@ public void parseStmt(Analyzer analyzer) throws IOException { try { stmt.analyze(analyzer); } catch (Exception e) { - LOG.warn("CreateMaterializedViewStmt analyze failed, reason=", e); + LOG.warn("CreateMaterializedViewStmt analyze failed, mv=" + defineStmt.originStmt + ", reason=", e); return; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 2b3f82a312cb37f..a5988dd71ae949b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -47,6 +47,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.Util; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; import org.apache.doris.resource.Tag; import org.apache.doris.statistics.AnalysisInfo; @@ -55,6 +56,8 @@ import org.apache.doris.statistics.HistogramTask; import org.apache.doris.statistics.MVAnalysisTask; import org.apache.doris.statistics.OlapAnalysisTask; +import org.apache.doris.statistics.TableStats; +import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TColumn; @@ -1121,6 +1124,32 @@ public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { return new MVAnalysisTask(info); } + @Override + public boolean needReAnalyzeTable(TableStats tblStats) { + long rowCount = getRowCount(); + // TODO: Do we need to analyze an empty table? + if (rowCount == 0) { + return false; + } + long updateRows = tblStats.updatedRows.get(); + int tblHealth = StatisticsUtil.getTableHealth(rowCount, updateRows); + return tblHealth < Config.table_stats_health_threshold; + } + + @Override + public Set findReAnalyzeNeededPartitions(TableStats tableStats) { + if (tableStats == null) { + return getPartitionNames().stream().map(this::getPartition) + .filter(Partition::hasData).map(Partition::getName).collect(Collectors.toSet()); + } + return getPartitionNames().stream() + .map(this::getPartition) + .filter(Partition::hasData) + .filter(partition -> + partition.getVisibleVersionTime() >= tableStats.updatedTime).map(Partition::getName) + .collect(Collectors.toSet()); + } + @Override public long getRowCount() { long rowCount = 0; @@ -1548,7 +1577,8 @@ public long getReplicaCount() { public void checkNormalStateForAlter() throws DdlException { if (state != OlapTableState.NORMAL) { - throw new DdlException("Table[" + name + "]'s state is not NORMAL. Do not allow doing ALTER ops"); + throw new DdlException("Table[" + name + "]'s state(" + state.toString() + + ") is not NORMAL. Do not allow doing ALTER ops"); } } @@ -1804,6 +1834,7 @@ public void setStoragePolicy(String storagePolicy) throws UserException { TableProperty tableProperty = getOrCreatTableProperty(); tableProperty.modifyTableProperties(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY, storagePolicy); tableProperty.buildStoragePolicy(); + partitionInfo.refreshTableStoragePolicy(storagePolicy); } public String getStoragePolicy() { @@ -2262,4 +2293,19 @@ public void getColumnDesc(long selectedIndexId, List columnsDesc, List< } } } + + @Override + public void analyze(String dbName) { + for (MaterializedIndexMeta meta : indexIdToMeta.values()) { + try { + ConnectContext connectContext = new ConnectContext(); + connectContext.setCluster(SystemInfoService.DEFAULT_CLUSTER); + connectContext.setDatabase(dbName); + Analyzer analyzer = new Analyzer(Env.getCurrentEnv(), connectContext); + meta.parseStmt(analyzer); + } catch (IOException e) { + e.printStackTrace(); + } + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java index 6fad5e3fe340cc1..53eb6597358b207 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java @@ -253,6 +253,11 @@ public List getMaterializedIndices(IndexExtState extState) { return indices; } + public long getAllDataSize() { + return getDataSize() + getRemoteDataSize(); + } + + // this is local data size public long getDataSize() { long dataSize = 0; for (MaterializedIndex mIndex : getMaterializedIndices(IndexExtState.VISIBLE)) { 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 e57cad6cb6388cc..d319882af43f033 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 @@ -230,6 +230,13 @@ public void setDataProperty(long partitionId, DataProperty newDataProperty) { idToDataProperty.put(partitionId, newDataProperty); } + public void refreshTableStoragePolicy(String storagePolicy) { + idToStoragePolicy.replaceAll((k, v) -> storagePolicy); + idToDataProperty.entrySet().forEach(entry -> { + entry.getValue().setStoragePolicy(storagePolicy); + }); + } + public String getStoragePolicy(long partitionId) { return idToStoragePolicy.getOrDefault(partitionId, ""); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java index d5035dbd52d3b9d..7215cf0fc7634f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java @@ -250,10 +250,10 @@ public class SchemaTable extends Table { // Compatible with mysql for mysqldump .put("column_statistics", new SchemaTable(SystemIdGenerator.getNextId(), "column_statistics", TableType.SCHEMA, - builder().column("SCHEMA_NAME", ScalarType.createVarchar(64)) - .column("TABLE_NAME", ScalarType.createVarchar(64)) - .column("COLUMN_NAME", ScalarType.createVarchar(64)) - .column("HISTOGRAM", ScalarType.createJsonbType()).build())) + builder().column("SCHEMA_NAME", ScalarType.createVarchar(64)) + .column("TABLE_NAME", ScalarType.createVarchar(64)) + .column("COLUMN_NAME", ScalarType.createVarchar(64)) + .column("HISTOGRAM", ScalarType.createJsonbType()).build())) .put("files", new SchemaTable(SystemIdGenerator.getNextId(), "files", TableType.SCHEMA, builder().column("FILE_ID", ScalarType.createType(PrimitiveType.BIGINT)) @@ -384,37 +384,66 @@ public class SchemaTable extends Table { .column("COLLATION_CONNECTION", ScalarType.createVarchar(32)) .column("DATABASE_COLLATION", ScalarType.createVarchar(32)).build())) .put("rowsets", new SchemaTable(SystemIdGenerator.getNextId(), "rowsets", TableType.SCHEMA, - builder().column("BACKEND_ID", ScalarType.createType(PrimitiveType.BIGINT)) - .column("ROWSET_ID", ScalarType.createVarchar(64)) - .column("TABLET_ID", ScalarType.createType(PrimitiveType.BIGINT)) - .column("ROWSET_NUM_ROWS", ScalarType.createType(PrimitiveType.BIGINT)) - .column("TXN_ID", ScalarType.createType(PrimitiveType.BIGINT)) - .column("NUM_SEGMENTS", ScalarType.createType(PrimitiveType.BIGINT)) - .column("START_VERSION", ScalarType.createType(PrimitiveType.BIGINT)) - .column("END_VERSION", ScalarType.createType(PrimitiveType.BIGINT)) - .column("INDEX_DISK_SIZE", ScalarType.createType(PrimitiveType.BIGINT)) - .column("DATA_DISK_SIZE", ScalarType.createType(PrimitiveType.BIGINT)) - .column("CREATION_TIME", ScalarType.createType(PrimitiveType.BIGINT)) - .column("NEWEST_WRITE_TIMESTAMP", ScalarType.createType(PrimitiveType.BIGINT)) - .build())) + builder().column("BACKEND_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("ROWSET_ID", ScalarType.createVarchar(64)) + .column("TABLET_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("ROWSET_NUM_ROWS", ScalarType.createType(PrimitiveType.BIGINT)) + .column("TXN_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("NUM_SEGMENTS", ScalarType.createType(PrimitiveType.BIGINT)) + .column("START_VERSION", ScalarType.createType(PrimitiveType.BIGINT)) + .column("END_VERSION", ScalarType.createType(PrimitiveType.BIGINT)) + .column("INDEX_DISK_SIZE", ScalarType.createType(PrimitiveType.BIGINT)) + .column("DATA_DISK_SIZE", ScalarType.createType(PrimitiveType.BIGINT)) + .column("CREATION_TIME", ScalarType.createType(PrimitiveType.BIGINT)) + .column("NEWEST_WRITE_TIMESTAMP", ScalarType.createType(PrimitiveType.BIGINT)) + .build())) .put("parameters", new SchemaTable(SystemIdGenerator.getNextId(), "parameters", TableType.SCHEMA, - builder().column("SPECIFIC_CATALOG", ScalarType.createVarchar(64)) - .column("SPECIFIC_SCHEMA", ScalarType.createVarchar(64)) - .column("SPECIFIC_NAME", ScalarType.createVarchar(64)) - .column("ORDINAL_POSITION", ScalarType.createVarchar(77)) - .column("PARAMETER_MODE", ScalarType.createVarchar(77)) - .column("PARAMETER_NAME", ScalarType.createVarchar(77)) - .column("DATA_TYPE", ScalarType.createVarchar(64)) - .column("CHARACTER_OCTET_LENGTH", ScalarType.createVarchar(64)) - .column("NUMERIC_PRECISION", ScalarType.createVarchar(512)) - .column("NUMERIC_SCALE", ScalarType.createVarchar(64)) - .column("DATETIME_PRECISION", ScalarType.createVarchar(64)) - .column("CHARACTER_SET_NAME", ScalarType.createVarchar(256)) - .column("COLLATION_NAME", ScalarType.createVarchar(64)) - .column("DTD_IDENTIFIER", ScalarType.createVarchar(64)) - .column("ROUTINE_TYPE", ScalarType.createVarchar(64)) - .column("DATA_TYPEDTD_IDENDS", ScalarType.createVarchar(64)) - .build())) + builder().column("SPECIFIC_CATALOG", ScalarType.createVarchar(64)) + .column("SPECIFIC_SCHEMA", ScalarType.createVarchar(64)) + .column("SPECIFIC_NAME", ScalarType.createVarchar(64)) + .column("ORDINAL_POSITION", ScalarType.createVarchar(77)) + .column("PARAMETER_MODE", ScalarType.createVarchar(77)) + .column("PARAMETER_NAME", ScalarType.createVarchar(77)) + .column("DATA_TYPE", ScalarType.createVarchar(64)) + .column("CHARACTER_OCTET_LENGTH", ScalarType.createVarchar(64)) + .column("NUMERIC_PRECISION", ScalarType.createVarchar(512)) + .column("NUMERIC_SCALE", ScalarType.createVarchar(64)) + .column("DATETIME_PRECISION", ScalarType.createVarchar(64)) + .column("CHARACTER_SET_NAME", ScalarType.createVarchar(256)) + .column("COLLATION_NAME", ScalarType.createVarchar(64)) + .column("DTD_IDENTIFIER", ScalarType.createVarchar(64)) + .column("ROUTINE_TYPE", ScalarType.createVarchar(64)) + .column("DATA_TYPEDTD_IDENDS", ScalarType.createVarchar(64)) + .build())) + .put("metadata_name_ids", new SchemaTable(SystemIdGenerator.getNextId(), + "metadata_name_ids", TableType.SCHEMA, + builder().column("CATALOG_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("CATALOG_NAME", ScalarType.createVarchar(FN_REFLEN)) + .column("DATABASE_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("DATABASE_NAME", ScalarType.createVarchar(NAME_CHAR_LEN)) + .column("TABLE_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("TABLE_NAME", ScalarType.createVarchar(NAME_CHAR_LEN)) + .build())) + .put("profiling", new SchemaTable(SystemIdGenerator.getNextId(), "profiling", TableType.SCHEMA, + builder().column("QUERY_ID", ScalarType.createType(PrimitiveType.INT)) + .column("SEQ", ScalarType.createType(PrimitiveType.INT)) + .column("STATE", ScalarType.createVarchar(30)) + .column("DURATION", ScalarType.createType(PrimitiveType.DOUBLE)) + .column("CPU_USER", ScalarType.createType(PrimitiveType.DOUBLE)) + .column("CPU_SYSTEM", ScalarType.createType(PrimitiveType.DOUBLE)) + .column("CONTEXT_VOLUNTARY", ScalarType.createType(PrimitiveType.INT)) + .column("CONTEXT_INVOLUNTARY", ScalarType.createType(PrimitiveType.INT)) + .column("BLOCK_OPS_IN", ScalarType.createType(PrimitiveType.INT)) + .column("BLOCK_OPS_OUT", ScalarType.createType(PrimitiveType.INT)) + .column("MESSAGES_SENT", ScalarType.createType(PrimitiveType.INT)) + .column("MESSAGES_RECEIVED", ScalarType.createType(PrimitiveType.INT)) + .column("PAGE_FAULTS_MAJOR", ScalarType.createType(PrimitiveType.INT)) + .column("PAGE_FAULTS_MINOR", ScalarType.createType(PrimitiveType.INT)) + .column("SWAPS", ScalarType.createType(PrimitiveType.INT)) + .column("SOURCE_FUNCTION", ScalarType.createVarchar(30)) + .column("SOURCE_FILE", ScalarType.createVarchar(20)) + .column("SOURCE_LINE", ScalarType.createType(PrimitiveType.INT)) + .build())) .build(); protected SchemaTable(long id, String name, TableType type, List baseSchema) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java index 98ebe54b960a411..453a50ad115045b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java @@ -37,7 +37,12 @@ import org.apache.logging.log4j.Logger; import java.io.File; +import java.util.Collections; import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Spark resource for etl or query. @@ -159,16 +164,20 @@ public Map getSparkConfigs() { } public Map getEnvConfigsWithoutPrefix() { - Map envConfig = Maps.newHashMap(); - if (envConfigs != null) { - for (Map.Entry entry : envConfigs.entrySet()) { - if (entry.getKey().startsWith(ENV_PREFIX)) { - String key = entry.getKey().substring(ENV_PREFIX.length()); - envConfig.put(key, entry.getValue()); - } - } - } - return envConfig; + return Stream.concat( + getSystemEnvConfigs().entrySet().stream(), + Optional.ofNullable(envConfigs).orElse(Collections.emptyMap()).entrySet().stream() + ) + .filter(entry -> entry.getKey().startsWith(ENV_PREFIX)) + .collect(Collectors.toMap( + entry -> entry.getKey().substring(ENV_PREFIX.length()), + Entry::getValue, + (oldValue, newValue) -> newValue + )); + } + + public Map getSystemEnvConfigs() { + return System.getenv(); } public Pair getYarnResourcemanagerAddressPair() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index c05d743a7b4fc4f..12689894b4835ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -30,6 +30,7 @@ import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.TableStats; import org.apache.doris.thrift.TTableDescriptor; import com.google.common.base.Preconditions; @@ -570,4 +571,16 @@ public DatabaseIf getDatabase() { public Optional getColumnStatistic(String colName) { return Optional.empty(); } + + public void analyze(String dbName) {} + + @Override + public boolean needReAnalyzeTable(TableStats tblStats) { + return true; + } + + @Override + public Set findReAnalyzeNeededPartitions(TableStats tableStats) { + return Collections.emptySet(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java index 78717f0eca769d6..21e2ddd154b9ee3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java @@ -23,6 +23,7 @@ import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.TableStats; import org.apache.doris.thrift.TTableDescriptor; import com.google.common.collect.Lists; @@ -136,6 +137,10 @@ default int getBaseColumnIdxByName(String colName) { Optional getColumnStatistic(String colName); + boolean needReAnalyzeTable(TableStats tblStats); + + Set findReAnalyzeNeededPartitions(TableStats tableStats); + void write(DataOutput out) throws IOException; /** @@ -145,7 +150,7 @@ enum TableType { MYSQL, ODBC, OLAP, SCHEMA, INLINE_VIEW, VIEW, BROKER, ELASTICSEARCH, HIVE, ICEBERG, @Deprecated HUDI, JDBC, TABLE_VALUED_FUNCTION, HMS_EXTERNAL_TABLE, ES_EXTERNAL_TABLE, MATERIALIZED_VIEW, JDBC_EXTERNAL_TABLE, ICEBERG_EXTERNAL_TABLE, TEST_EXTERNAL_TABLE, PAIMON_EXTERNAL_TABLE, MAX_COMPUTE_EXTERNAL_TABLE, - HUDI_EXTERNAL_TABLE; + HUDI_EXTERNAL_TABLE, DELTALAKE_EXTERNAL_TABLE; public String toEngineName() { switch (this) { @@ -182,6 +187,8 @@ public String toEngineName() { return "iceberg"; case HUDI_EXTERNAL_TABLE: return "hudi"; + case DELTALAKE_EXTERNAL_TABLE: + return "deltalake"; default: return null; } @@ -210,6 +217,7 @@ public String toMysqlType() { case ES_EXTERNAL_TABLE: case ICEBERG_EXTERNAL_TABLE: case PAIMON_EXTERNAL_TABLE: + case DELTALAKE_EXTERNAL_TABLE: return "EXTERNAL TABLE"; default: return null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/DeltaLakeExternalDataBase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/DeltaLakeExternalDataBase.java new file mode 100644 index 000000000000000..2db5c4eb8357184 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/DeltaLakeExternalDataBase.java @@ -0,0 +1,34 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.catalog.external; + +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.InitDatabaseLog; +import org.apache.doris.datasource.deltalake.DeltaLakeExternalCatalog; + +public class DeltaLakeExternalDataBase extends HMSExternalDatabase { + + public DeltaLakeExternalDataBase(ExternalCatalog extCatalog, long id, String name) { + super(extCatalog, id, name, InitDatabaseLog.Type.DELTALAKE); + } + + @Override + protected DeltaLakeExternalTable getExternalTable(String tableName, long tblId, ExternalCatalog catalog) { + return new DeltaLakeExternalTable(tblId, tableName, name, (DeltaLakeExternalCatalog) extCatalog); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/DeltaLakeExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/DeltaLakeExternalTable.java new file mode 100644 index 000000000000000..cd5bb162001bc1c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/DeltaLakeExternalTable.java @@ -0,0 +1,159 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.catalog.external; + +import org.apache.doris.catalog.ArrayType; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.HiveMetaStoreClientHelper; +import org.apache.doris.catalog.MapType; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.catalog.StructType; +import org.apache.doris.catalog.Type; +import org.apache.doris.datasource.HMSExternalCatalog; +import org.apache.doris.datasource.deltalake.DeltaLakeExternalCatalog; + +import com.google.common.collect.Lists; +import io.delta.standalone.DeltaLog; +import io.delta.standalone.actions.Metadata; +import io.delta.standalone.types.DataType; +import io.delta.standalone.types.StructField; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.api.FieldSchema; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class DeltaLakeExternalTable extends HMSExternalTable { + public DeltaLakeExternalTable(long id, String name, String dbName, + DeltaLakeExternalCatalog catalog) { + super(id, name, dbName, catalog, TableType.DELTALAKE_EXTERNAL_TABLE); + } + + @Override + protected synchronized void makeSureInitialized() { + super.makeSureInitialized(); + if (!objectCreated) { + remoteTable = ((HMSExternalCatalog) catalog).getClient().getTable(dbName, name); + if (remoteTable == null) { + dlaType = DLAType.UNKNOWN; + } else { + if (supportedDeltaLakeTable()) { + dlaType = DLAType.DELTALAKE; + } else { + dlaType = DLAType.UNKNOWN; + } + } + objectCreated = true; + } + } + + private boolean supportedDeltaLakeTable() { + Map parameters = remoteTable.getParameters(); + if (parameters == null) { + return false; + } + // Check that the 'spark.sql.sources.provider' parameter exists and has a value of 'delta' + return "delta".equalsIgnoreCase(parameters.get("spark.sql.sources.provider")); + } + + @Override + public List initSchema() { + makeSureInitialized(); + List columns; + List schema = ((DeltaLakeExternalCatalog) catalog).getClient().getSchema(dbName, name); + io.delta.standalone.types.StructType deltaSchema = getDeltaTableSchema(this); + List tmpSchema = Lists.newArrayListWithCapacity(schema.size()); + for (StructField field : deltaSchema.getFields()) { + String columnName = field.getName(); + tmpSchema.add(new Column(columnName, fromDeltaTypeToDorisType(field.getDataType()), + true, null, true, null, "", true, null, -1, null)); + } + columns = tmpSchema; + initPartitionColumns(columns); + return columns; + } + + private static io.delta.standalone.types.StructType getDeltaTableSchema(DeltaLakeExternalTable table) { + String path = table.getRemoteTable().getSd().getLocation(); + Configuration conf = HiveMetaStoreClientHelper.getConfiguration(table); + DeltaLog deltaLog = DeltaLog.forTable(conf, path); + Metadata metadata = deltaLog.snapshot().getMetadata(); + io.delta.standalone.types.StructType tableSchema = metadata.getSchema(); + return tableSchema; + } + + private static Type fromDeltaTypeToDorisType(DataType dataType) { + String typeName = dataType.getTypeName(); + switch (typeName) { + case "boolean": + return Type.BOOLEAN; + case "byte": + case "tinyint": + return Type.TINYINT; + case "smallint": + return Type.SMALLINT; + case "integer": + return Type.INT; + case "long": + return Type.BIGINT; + case "float": + return Type.FLOAT; + case "double": + return Type.DOUBLE; + case "date": + return Type.DATEV2; + case "timestamp": + return ScalarType.createDatetimeV2Type(6); + case "string": + return Type.STRING; + case "decimal": + int precision = ((io.delta.standalone.types.DecimalType) dataType).getPrecision(); + int scale = ((io.delta.standalone.types.DecimalType) dataType).getScale(); + return ScalarType.createDecimalV3Type(precision, scale); + case "array": + io.delta.standalone.types.ArrayType arrayType = (io.delta.standalone.types.ArrayType) dataType; + Type innerType = fromDeltaTypeToDorisType(arrayType.getElementType()); + return ArrayType.create(innerType, true); + case "map": + io.delta.standalone.types.MapType mapType = (io.delta.standalone.types.MapType) dataType; + return new MapType(Type.STRING, fromDeltaTypeToDorisType(mapType.getValueType())); + case "struct": + io.delta.standalone.types.StructType deltaStructType = (io.delta.standalone.types.StructType) dataType; + ArrayList dorisFields = new ArrayList<>(); + for (io.delta.standalone.types.StructField deltaField : deltaStructType.getFields()) { + // Convert the Delta field type to a Doris type + Type dorisFieldType = fromDeltaTypeToDorisType(deltaField.getDataType()); + + // Create a Doris struct field with the same name and type + org.apache.doris.catalog.StructField dorisField = new org.apache.doris.catalog.StructField( + deltaField.getName(), dorisFieldType); + + // Add the Doris field to the list + dorisFields.add(dorisField); + } + // Create a Doris struct type with the converted fields + return new StructType(dorisFields); + case "null": + return Type.NULL; + case "binary": + default: + return Type.UNSUPPORTED; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java index 0f9ec3f56433c73..6f31ac18d77ddbc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java @@ -35,8 +35,10 @@ import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.TableStats; import org.apache.doris.thrift.TTableDescriptor; +import com.google.common.collect.Sets; import com.google.gson.annotations.SerializedName; import lombok.Getter; import org.apache.commons.lang3.NotImplementedException; @@ -46,8 +48,10 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.HashSet; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -375,4 +379,19 @@ public void gsonPostProcess() throws IOException { rwLock = new ReentrantReadWriteLock(true); objectCreated = false; } + + @Override + public boolean needReAnalyzeTable(TableStats tblStats) { + // TODO: Find a way to decide if this external table need to be reanalyzed. + // For now, simply return true for all external tables. + return true; + } + + @Override + public Set findReAnalyzeNeededPartitions(TableStats tableStats) { + HashSet partitions = Sets.newHashSet(); + // TODO: Find a way to collect external table partitions that need to be analyzed. + partitions.add("Dummy Partition"); + return partitions; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java index d75f86bd0883dac..318ea06f3413c2e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java @@ -46,6 +46,10 @@ public HMSExternalDatabase(ExternalCatalog extCatalog, long id, String name) { super(extCatalog, id, name, InitDatabaseLog.Type.HMS); } + public HMSExternalDatabase(ExternalCatalog extCatalog, long id, String name, InitDatabaseLog.Type type) { + super(extCatalog, id, name, type); + } + @Override protected HMSExternalTable getExternalTable(String tableName, long tblId, ExternalCatalog catalog) { return new HMSExternalTable(tblId, tableName, name, (HMSExternalCatalog) extCatalog); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java index a4c19ecc4534915..c1de1ea98d7555f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java @@ -102,16 +102,16 @@ public class HMSExternalTable extends ExternalTable { SUPPORTED_HUDI_FILE_FORMATS.add("com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat"); } - private volatile org.apache.hadoop.hive.metastore.api.Table remoteTable = null; - private List partitionColumns; + protected volatile org.apache.hadoop.hive.metastore.api.Table remoteTable = null; + protected List partitionColumns; - private DLAType dlaType = DLAType.UNKNOWN; + protected DLAType dlaType = DLAType.UNKNOWN; // No as precise as row count in TableStats, but better than none. private long estimatedRowCount = -1; public enum DLAType { - UNKNOWN, HIVE, HUDI, ICEBERG + UNKNOWN, HIVE, HUDI, ICEBERG, DELTALAKE } /** @@ -126,6 +126,10 @@ public HMSExternalTable(long id, String name, String dbName, HMSExternalCatalog super(id, name, catalog, dbName, TableType.HMS_EXTERNAL_TABLE); } + public HMSExternalTable(long id, String name, String dbName, HMSExternalCatalog catalog, TableType type) { + super(id, name, catalog, dbName, type); + } + public boolean isSupportedHmsTable() { makeSureInitialized(); return dlaType != DLAType.UNKNOWN; @@ -149,7 +153,7 @@ protected synchronized void makeSureInitialized() { } } objectCreated = true; - estimatedRowCount = getRowCountFromExternalSource(); + estimatedRowCount = getRowCountFromExternalSource(true); } } @@ -273,7 +277,7 @@ public long getUpdateTime() { @Override public long getRowCount() { makeSureInitialized(); - long rowCount = getRowCountFromExternalSource(); + long rowCount = getRowCountFromExternalSource(false); if (rowCount == -1) { LOG.debug("Will estimate row count from file list."); rowCount = StatisticsUtil.getRowCountFromFileList(this); @@ -281,11 +285,11 @@ public long getRowCount() { return rowCount; } - private long getRowCountFromExternalSource() { + private long getRowCountFromExternalSource(boolean isInit) { long rowCount; switch (dlaType) { case HIVE: - rowCount = StatisticsUtil.getHiveRowCount(this); + rowCount = StatisticsUtil.getHiveRowCount(this, isInit); break; case ICEBERG: rowCount = StatisticsUtil.getIcebergRowCount(this); @@ -465,7 +469,7 @@ private List getIcebergSchema(List hmsSchema) { return tmpSchema; } - private void initPartitionColumns(List schema) { + protected void initPartitionColumns(List schema) { List partitionKeys = remoteTable.getPartitionKeys().stream().map(FieldSchema::getName) .collect(Collectors.toList()); partitionColumns = Lists.newArrayListWithCapacity(partitionKeys.size()); 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 a368e33a774faa7..191cc529bf9c57b 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 @@ -210,7 +210,7 @@ private static int getBucketsNum(DynamicPartitionProperty property, OlapTable ta ArrayList partitionSizeArray = Lists.newArrayList(); for (Partition partition : partitions) { if (partition.getVisibleVersion() >= 2) { - partitionSizeArray.add(partition.getDataSize()); + partitionSizeArray.add(partition.getAllDataSize()); } } @@ -501,11 +501,12 @@ private void executeDynamicPartition(Collection> dynamicPartiti olapTable = (OlapTable) db.getTableNullable(tableId); // Only OlapTable has DynamicPartitionProperty if (olapTable == null - || olapTable.isBeingSynced() || !olapTable.dynamicPartitionExists() || !olapTable.getTableProperty().getDynamicPartitionProperty().getEnable()) { iterator.remove(); continue; + } else if (olapTable.isBeingSynced()) { + continue; } olapTable.readLock(); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java index a44e75672dde984..95cbbc348e3aca2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java @@ -70,6 +70,8 @@ public class PropertyAnalyzer { public static final String PROPERTIES_VERSION_INFO = "version_info"; // for restore public static final String PROPERTIES_SCHEMA_VERSION = "schema_version"; + public static final String PROPERTIES_PARTITION_ID = "partition_id"; + public static final String PROPERTIES_VISIBLE_VERSION = "visible_version"; public static final String PROPERTIES_BF_COLUMNS = "bloom_filter_columns"; public static final String PROPERTIES_BF_FPP = "bloom_filter_fpp"; @@ -427,6 +429,30 @@ public static int analyzeSchemaVersion(Map properties) throws An return schemaVersion; } + private static Long getPropertyLong(Map properties, String propertyId) throws AnalysisException { + long id = -1; + if (properties != null && properties.containsKey(propertyId)) { + String propertyIdStr = properties.get(propertyId); + try { + id = Long.parseLong(propertyIdStr); + } catch (Exception e) { + throw new AnalysisException("Invalid property long id: " + propertyIdStr); + } + + properties.remove(propertyId); + } + + return id; + } + + public static Long analyzePartitionId(Map properties) throws AnalysisException { + return getPropertyLong(properties, PROPERTIES_PARTITION_ID); + } + + public static Long analyzeVisibleVersion(Map properties) throws AnalysisException { + return getPropertyLong(properties, PROPERTIES_VISIBLE_VERSION); + } + public static Set analyzeBloomFilterColumns(Map properties, List columns, KeysType keysType) throws AnalysisException { Set bfColumns = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogFactory.java index 5aa75daaad93645..07a03ed86168280 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogFactory.java @@ -27,6 +27,7 @@ import org.apache.doris.catalog.Resource; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; +import org.apache.doris.datasource.deltalake.DeltaLakeExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergExternalCatalogFactory; import org.apache.doris.datasource.jdbc.JdbcExternalCatalog; import org.apache.doris.datasource.paimon.PaimonExternalCatalogFactory; @@ -130,6 +131,9 @@ private static CatalogIf createCatalog(long catalogId, String name, String resou case "max_compute": catalog = new MaxComputeExternalCatalog(catalogId, name, resource, props, comment); break; + case "deltalake": + catalog = new DeltaLakeExternalCatalog(catalogId, name, resource, props, comment); + break; case "test": if (!FeConstants.runningUnitTest) { throw new DdlException("test catalog is only for FE unit test"); @@ -142,7 +146,15 @@ private static CatalogIf createCatalog(long catalogId, String name, String resou if (!isReplay) { // set some default properties when creating catalog. // do not call this method when replaying edit log. Because we need to keey the original properties. - catalog.setDefaultProps(); + catalog.setDefaultPropsWhenCreating(isReplay); + // This will check if the customized access controller can be created successfully. + // If failed, it will throw exception and the catalog will not be created. + try { + catalog.initAccessController(true); + } catch (Exception e) { + LOG.warn("Failed to init access controller", e); + throw new DdlException("Failed to init access controller: " + e.getMessage()); + } } return catalog; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java index b84c769eacff5e6..d135018e7541ba2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java @@ -170,4 +170,6 @@ default CatalogLog constructEditLog() { // Return a copy of all db collection. @SuppressWarnings({"rawtypes", "unchecked"}) public Collection getAllDbs(); + + public boolean enableAutoAnalyze(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index 9dbc40ff8476ca3..35d03dfabcfbf44 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Resource; +import org.apache.doris.catalog.external.DeltaLakeExternalDataBase; import org.apache.doris.catalog.external.EsExternalDatabase; import org.apache.doris.catalog.external.ExternalDatabase; import org.apache.doris.catalog.external.ExternalTable; @@ -71,6 +72,8 @@ public abstract class ExternalCatalog implements CatalogIf>, Writable, GsonPostProcessable { private static final Logger LOG = LogManager.getLogger(ExternalCatalog.class); + public static final String ENABLE_AUTO_ANALYZE = "enable.auto.analyze"; + // Unique id of this catalog, will be assigned after catalog is loaded. @SerializedName(value = "id") protected long id; @@ -109,7 +112,7 @@ protected List listDatabaseNames() { + "listDatabaseNames from remote client when init catalog with " + logType.name()); } - public void setDefaultProps() { + public void setDefaultPropsWhenCreating(boolean isReplay) throws DdlException { // set some default properties when creating catalog } @@ -202,8 +205,10 @@ public void checkProperties() throws DdlException { * "access_controller.properties.prop1" = "xxx", * "access_controller.properties.prop2" = "yyy", * ) + *

+ * isDryRun: if true, it will try to create the custom access controller, but will not add it to the access manager. */ - public void initAccessController() { + public void initAccessController(boolean isDryRun) { Map properties = getCatalogProperty().getProperties(); // 1. get access controller class name String className = properties.getOrDefault(CatalogMgr.ACCESS_CONTROLLER_CLASS_PROP, ""); @@ -223,7 +228,7 @@ public void initAccessController() { } // 3. create access controller - Env.getCurrentEnv().getAccessManager().createAccessController(name, className, acProperties); + Env.getCurrentEnv().getAccessManager().createAccessController(name, className, acProperties, isDryRun); } // init schema related objects @@ -483,6 +488,8 @@ protected ExternalDatabase getDbForInit(String dbName, return new TestExternalDatabase(this, dbId, dbName); case PAIMON: return new PaimonExternalDatabase(this, dbId, dbName); + case DELTALAKE: + return new DeltaLakeExternalDataBase(this, dbId, dbName); default: break; } @@ -582,6 +589,20 @@ public boolean useSelfSplitter() { @Override public Collection getAllDbs() { + makeSureInitialized(); return new HashSet<>(idToDb.values()); } + + @Override + public boolean enableAutoAnalyze() { + // By default, external catalog disables auto analyze, uses could set catalog property to enable it: + // "enable.auto.analyze" = true + Map properties = catalogProperty.getProperties(); + boolean ret = false; + if (properties.containsKey(ENABLE_AUTO_ANALYZE) + && properties.get(ENABLE_AUTO_ANALYZE).equalsIgnoreCase("true")) { + ret = true; + } + return ret; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index 462fd3527a8a97a..8bdea35e45c63ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -78,6 +78,13 @@ public HMSExternalCatalog(long catalogId, String name, String resource, Map props, + String comment, InitCatalogLog.Type type) { + super(catalogId, name, type, comment); + props = PropertyConverter.convertToMetaProperties(props); + catalogProperty = new CatalogProperty(resource, props); + } + @Override public void checkProperties() throws DdlException { super.checkProperties(); @@ -289,7 +296,10 @@ public void notifyPropertiesUpdated(Map updatedProps) { } @Override - public void setDefaultProps() { + public void setDefaultPropsWhenCreating(boolean isReplay) { + if (isReplay) { + return; + } if (catalogProperty.getOrDefault(PROP_ALLOW_FALLBACK_TO_SIMPLE_AUTH, "").isEmpty()) { // always allow fallback to simple auth, so to support both kerberos and simple auth catalogProperty.addProperty(PROP_ALLOW_FALLBACK_TO_SIMPLE_AUTH, "true"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InitCatalogLog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InitCatalogLog.java index dd30fbf43c95acc..e766324a726befe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InitCatalogLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InitCatalogLog.java @@ -40,6 +40,7 @@ public enum Type { PAIMON, MAX_COMPUTE, HUDI, + DELTALAKE, TEST, UNKNOWN; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InitDatabaseLog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InitDatabaseLog.java index 3a85fb1edc54c4c..1659a38cc030435 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InitDatabaseLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InitDatabaseLog.java @@ -40,6 +40,7 @@ public enum Type { MAX_COMPUTE, HUDI, PAIMON, + DELTALAKE, TEST, UNKNOWN; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 2be8e61fdf1098e..885e932a8653cf7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -2223,7 +2223,12 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep "Can not create UNIQUE KEY table that enables Merge-On-write" + " with storage policy(" + storagePolicy + ")"); } - olapTable.setStoragePolicy(storagePolicy); + // Consider one situation: if the table has no storage policy but some partitions + // have their own storage policy then it might be erased by the following function. + // So we only set the storage policy if the table's policy is not null or empty + if (!Strings.isNullOrEmpty(storagePolicy)) { + olapTable.setStoragePolicy(storagePolicy); + } TTabletType tabletType; try { @@ -3109,6 +3114,8 @@ public long loadDb(DataInputStream dis, long checksum) throws IOException, DdlEx idToDb.put(db.getId(), db); fullNameToDb.put(db.getFullName(), db); Env.getCurrentGlobalTransactionMgr().addDatabaseTransactionMgr(db.getId()); + + db.analyze(); } // ATTN: this should be done after load Db, and before loadAlterJob recreateTabletInvertIndex(); @@ -3132,4 +3139,9 @@ public void replayAutoIncrementIdUpdateLog(AutoIncrementIdUpdateLog log) throws OlapTable olapTable = (OlapTable) db.getTableOrMetaException(log.getTableId(), TableType.OLAP); olapTable.getAutoIncrementGenerator().applyChange(log.getColumnId(), log.getBatchEndId()); } + + @Override + public boolean enableAutoAnalyze() { + return true; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/deltalake/DeltaLakeExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/deltalake/DeltaLakeExternalCatalog.java new file mode 100644 index 000000000000000..af142ea731592a6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/deltalake/DeltaLakeExternalCatalog.java @@ -0,0 +1,69 @@ +// 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.datasource.deltalake; + +import org.apache.doris.catalog.external.HMSExternalDatabase; +import org.apache.doris.catalog.external.HMSExternalTable; +import org.apache.doris.datasource.HMSExternalCatalog; +import org.apache.doris.datasource.InitCatalogLog; +import org.apache.doris.datasource.SessionContext; + +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.Table; + +import java.util.List; +import java.util.Map; + +public class DeltaLakeExternalCatalog extends HMSExternalCatalog { + + public DeltaLakeExternalCatalog(long catalogId, String name, String resource, Map props, + String comment) { + super(catalogId, name, resource, props, comment, InitCatalogLog.Type.DELTALAKE); + } + + @Override + public List listTableNames(SessionContext ctx, String dbName) { + makeSureInitialized(); + HMSExternalDatabase hmsExternalDatabase = (HMSExternalDatabase) idToDb.get(dbNameToId.get(dbName)); + if (hmsExternalDatabase != null && hmsExternalDatabase.isInitialized()) { + List names = Lists.newArrayList(); + for (HMSExternalTable table : hmsExternalDatabase.getTables()) { + String tableName = table.getName(); + Table tableDetails = client.getTable(dbName, tableName); + Map parameters = tableDetails.getParameters(); + String provider = parameters.get("spark.sql.sources.provider"); + if ("delta".equalsIgnoreCase(provider)) { + names.add(tableName); + } + } + return names; + } else { + List allTableNames = client.getAllTables(getRealTableName(dbName)); + List deltaTableNames = Lists.newArrayList(); + for (String tableName : allTableNames) { + Table tableDetails = client.getTable(dbName, tableName); + Map parameters = tableDetails.getParameters(); + String provider = parameters.get("spark.sql.sources.provider"); + if ("delta".equalsIgnoreCase(provider)) { + deltaTableNames.add(tableName); + } + } + return deltaTableNames; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java index 753a5d6906c436c..7617ad71801a09c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java @@ -166,4 +166,16 @@ public boolean tableExist(SessionContext ctx, String dbName, String tblName) { makeSureInitialized(); return jdbcClient.isTableExist(dbName, tblName); } + + @Override + public void setDefaultPropsWhenCreating(boolean isReplay) throws DdlException { + if (isReplay) { + return; + } + Map properties = Maps.newHashMap(); + if (properties.containsKey(JdbcResource.DRIVER_URL) && !properties.containsKey(JdbcResource.CHECK_SUM)) { + properties.put(JdbcResource.CHECK_SUM, + JdbcResource.computeObjectChecksum(properties.get(JdbcResource.DRIVER_URL))); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcSQLServerClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcSQLServerClient.java index 0f4b287698a16b5..8657b381b099e85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcSQLServerClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcSQLServerClient.java @@ -81,6 +81,7 @@ protected Type jdbcTypeToDoris(JdbcFieldSchema fieldSchema) { case "ntext": case "time": case "datetimeoffset": + case "uniqueidentifier": return ScalarType.createStringType(); case "image": case "binary": diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/HardwareInfoController.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/HardwareInfoController.java index 8b3802adfa71072..087511654815a4d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/HardwareInfoController.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/HardwareInfoController.java @@ -36,6 +36,7 @@ import oshi.software.os.OSFileStore; import oshi.software.os.OSProcess; import oshi.software.os.OperatingSystem; +import oshi.software.os.OperatingSystem.ProcessSorting; import oshi.util.FormatUtil; import oshi.util.Util; @@ -101,8 +102,8 @@ private List getProcessor(CentralProcessor processor) { processorInfo.add(" " + processor.getPhysicalProcessorCount() + " physical CPU core(s)"); processorInfo.add(" " + processor.getLogicalProcessorCount() + " logical CPU(s)"); - processorInfo.add("Identifier:   " + processor.getIdentifier()); - processorInfo.add("ProcessorID:   " + processor.getProcessorID()); + processorInfo.add("Identifier:   " + processor.getProcessorIdentifier().getIdentifier()); + processorInfo.add("ProcessorID:   " + processor.getProcessorIdentifier().getProcessorID()); processorInfo.add("Context Switches/Interrupts:   " + processor.getContextSwitches() + " / " + processor.getInterrupts() + "
"); @@ -150,7 +151,7 @@ private List getProcessor(CentralProcessor processor) { procCpu.append(String.format(" %.1f%%", avg * 100)); } processorInfo.add(procCpu.toString()); - long freq = processor.getVendorFreq(); + long freq = processor.getProcessorIdentifier().getVendorFreq(); if (freq > 0) { processorInfo.add("Vendor Frequency:   " + FormatUtil.formatHertz(freq)); } @@ -187,7 +188,8 @@ private List getProcesses(OperatingSystem os, GlobalMemory memory) { processInfo.add("Processes:   " + os.getProcessCount() + ", Threads:   " + os.getThreadCount()); // Sort by highest CPU - List procs = Arrays.asList(os.getProcesses(5, OperatingSystem.ProcessSort.CPU)); + + List procs = os.getProcesses((osProcess) -> true, ProcessSorting.CPU_DESC, 5); processInfo.add("         PID %CPU %MEM VSZ RSS Name"); for (int i = 0; i < procs.size() && i < 5; i++) { @@ -201,7 +203,7 @@ private List getProcesses(OperatingSystem os, GlobalMemory memory) { return processInfo; } - private List getDisks(HWDiskStore[] diskStores) { + private List getDisks(List diskStores) { List diskInfo = new ArrayList<>(); diskInfo.add("Disks:  "); for (HWDiskStore disk : diskStores) { @@ -213,7 +215,7 @@ private List getDisks(HWDiskStore[] diskStores) { readwrite ? disk.getReads() : "?", readwrite ? FormatUtil.formatBytes(disk.getReadBytes()) : "?", readwrite ? disk.getWrites() : "?", readwrite ? FormatUtil.formatBytes(disk.getWriteBytes()) : "?", readwrite ? disk.getTransferTime() : "?")); - HWPartition[] partitions = disk.getPartitions(); + List partitions = disk.getPartitions(); for (HWPartition part : partitions) { diskInfo.add(String.format("        " + " |-- %s: %s (%s) Maj:Min=%d:%d, size: %s%s", part.getIdentification(), @@ -232,13 +234,13 @@ private List getFileSystem(FileSystem fileSystem) { fsInfo.add(String.format("    File Descriptors: %d/%d", fileSystem.getOpenFileDescriptors(), fileSystem.getMaxFileDescriptors())); - OSFileStore[] fsArray = fileSystem.getFileStores(); - for (OSFileStore fs : fsArray) { + List fsList = fileSystem.getFileStores(); + for (OSFileStore fs : fsList) { long usable = fs.getUsableSpace(); long total = fs.getTotalSpace(); fsInfo.add(String.format("        " + "%s (%s) [%s] %s of %s free (%.1f%%), %s of %s files free (%.1f%%) is %s " - + (fs.getLogicalVolume() != null && fs.getLogicalVolume().length() > 0 ? "[%s]" : "%s") + + (fs.getLogicalVolume() != null && !fs.getLogicalVolume().isEmpty() ? "[%s]" : "%s") + " and is mounted at %s", fs.getName(), fs.getDescription().isEmpty() ? "file system" : fs.getDescription(), fs.getType(), FormatUtil.formatBytes(usable), FormatUtil.formatBytes(fs.getTotalSpace()), 100d * usable / total, @@ -249,7 +251,7 @@ private List getFileSystem(FileSystem fileSystem) { return fsInfo; } - private List getNetworkInterfaces(NetworkIF[] networkIFs) { + private List getNetworkInterfaces(List networkIFs) { List getNetwork = new ArrayList<>(); getNetwork.add("Network interfaces:  "); for (NetworkIF net : networkIFs) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java index 8ed148df5bbabf6..35ce557cae60c4d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java @@ -85,7 +85,7 @@ public Object streamLoad(HttpServletRequest request, return executeWithoutPassword(request, response, db, table, true); } - @RequestMapping(path = "/api/_stream_load_with_sql", + @RequestMapping(path = "/api/_http_stream", method = RequestMethod.PUT) public Object streamLoadWithSql(HttpServletRequest request, HttpServletResponse response) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StmtExecutionAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StmtExecutionAction.java index 7bfc0eb41827c09..d9e07eb9b36b13d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StmtExecutionAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StmtExecutionAction.java @@ -113,7 +113,7 @@ public Object executeSQL(@PathVariable(value = NS_KEY) String ns, @PathVariable( stmtRequestBody.limit); ConnectContext.get().changeDefaultCatalog(ns); - ConnectContext.get().setDatabase(getFullDbName(dbName)); + ConnectContext.get().setDatabase(fullDbName); String streamHeader = request.getHeader("X-Doris-Stream"); boolean isStream = !("false".equalsIgnoreCase(streamHeader)); @@ -176,10 +176,7 @@ private ResponseEntity executeQuery(ActionAuthorizationInfo authInfo, boolean is return null; } return ResponseEntityBuilder.ok(resultSet.getResult()); - } catch (InterruptedException e) { - LOG.warn("failed to execute stmt", e); - return ResponseEntityBuilder.okWithCommonError("Failed to execute sql: " + e.getMessage()); - } catch (ExecutionException e) { + } catch (InterruptedException | ExecutionException e) { LOG.warn("failed to execute stmt", e); return ResponseEntityBuilder.okWithCommonError("Failed to execute sql: " + e.getMessage()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/util/StatementSubmitter.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/util/StatementSubmitter.java index 5817f1d252e9600..83c5e075f222a64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/util/StatementSubmitter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/util/StatementSubmitter.java @@ -73,7 +73,7 @@ public class StatementSubmitter { private static final String JDBC_DRIVER = "org.mariadb.jdbc.Driver"; private static final String DB_URL_PATTERN = "jdbc:mariadb://127.0.0.1:%d/%s"; - private ThreadPoolExecutor executor = ThreadPoolManager.newDaemonCacheThreadPool(2, "SQL submitter", true); + private final ThreadPoolExecutor executor = ThreadPoolManager.newDaemonCacheThreadPool(2, "SQL submitter", true); public Future submit(StmtContext queryCtx) { Worker worker = new Worker(ConnectContext.get(), queryCtx); @@ -81,8 +81,8 @@ public Future submit(StmtContext queryCtx) { } private static class Worker implements Callable { - private ConnectContext ctx; - private StmtContext queryCtx; + private final ConnectContext ctx; + private final StmtContext queryCtx; public Worker(ConnectContext ctx, StmtContext queryCtx) { this.ctx = ctx; @@ -104,7 +104,7 @@ public ExecutionResultSet call() throws Exception { stmt = conn.prepareStatement( queryCtx.stmt, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY); // set fetch size to 1 to enable streaming result set to avoid OOM. - ((PreparedStatement) stmt).setFetchSize(1000); + stmt.setFetchSize(1000); ResultSet rs = ((PreparedStatement) stmt).executeQuery(); if (queryCtx.isStream) { StreamResponseInf streamResponse = new JsonStreamResponse(queryCtx.response); @@ -123,8 +123,7 @@ public ExecutionResultSet call() throws Exception { streamResponse.handleDdlAndExport(startTime); return new ExecutionResultSet(null); } - ExecutionResultSet resultSet = generateExecStatus(startTime); - return resultSet; + return generateExecStatus(startTime); } else { throw new Exception("Unsupported statement type"); } 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 34a6e2c77cdd3a4..179015b2bd073e4 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 @@ -107,7 +107,9 @@ import org.apache.doris.persist.ReplaceTableOperationLog; import org.apache.doris.persist.ReplicaPersistInfo; import org.apache.doris.persist.RoutineLoadOperation; +import org.apache.doris.persist.SetPartitionVersionOperationLog; import org.apache.doris.persist.SetReplicaStatusOperationLog; +import org.apache.doris.persist.SetTableStatusOperationLog; import org.apache.doris.persist.TableAddOrDropColumnsInfo; import org.apache.doris.persist.TableAddOrDropInvertedIndicesInfo; import org.apache.doris.persist.TableInfo; @@ -436,6 +438,11 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } + case OperationType.OP_SET_TABLE_STATUS: { + data = SetTableStatusOperationLog.read(in); + isRead = true; + break; + } case OperationType.OP_CREATE_REPOSITORY: { data = Repository.read(in); isRead = true; @@ -614,6 +621,11 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } + case OperationType.OP_SET_PARTITION_VERSION: { + data = SetPartitionVersionOperationLog.read(in); + isRead = true; + break; + } case OperationType.OP_DYNAMIC_PARTITION: case OperationType.OP_MODIFY_IN_MEMORY: case OperationType.OP_MODIFY_REPLICATION_NUM: diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java index bd975bb6fe568ff..5b0278041d9a74b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java @@ -156,6 +156,7 @@ public class ExportJob implements Writable { private TableRef tableRef; private Expr whereExpr; + private String whereSql; private String sql = ""; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java index 1f84d96bf3a47aa..6dd8675b3b34d5e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java @@ -208,7 +208,7 @@ private void createLoadingTask(Database db, BrokerPendingTaskAttachment attachme // Generate loading task and init the plan of task LoadLoadingTask task = new LoadLoadingTask(db, table, brokerDesc, brokerFileGroups, getDeadlineMs(), getExecMemLimit(), - isStrictMode(), transactionId, this, getTimeZone(), getTimeout(), + isStrictMode(), isPartialUpdate(), transactionId, this, getTimeZone(), getTimeout(), getLoadParallelism(), getSendBatchParallelism(), getMaxFilterRatio() <= 0, enableProfile ? jobProfile : null, isSingleTabletLoadPerSink(), useNewLoadScanNode(), getPriority()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java index 4c8528abd16b266..2532b0d80e8cd92 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java @@ -430,6 +430,7 @@ private void initDefaultJobProperties() { jobProperties.put(LoadStmt.EXEC_MEM_LIMIT, 2 * 1024 * 1024 * 1024L); jobProperties.put(LoadStmt.MAX_FILTER_RATIO_PROPERTY, 0.0); jobProperties.put(LoadStmt.STRICT_MODE, false); + jobProperties.put(LoadStmt.PARTIAL_COLUMNS, false); jobProperties.put(LoadStmt.TIMEZONE, TimeUtils.DEFAULT_TIME_ZONE); jobProperties.put(LoadStmt.LOAD_PARALLELISM, Config.default_load_parallelism); jobProperties.put(LoadStmt.SEND_BATCH_PARALLELISM, 1); @@ -1217,6 +1218,10 @@ protected boolean isStrictMode() { return (boolean) jobProperties.get(LoadStmt.STRICT_MODE); } + protected boolean isPartialUpdate() { + return (boolean) jobProperties.get(LoadStmt.PARTIAL_COLUMNS); + } + protected String getTimeZone() { return (String) jobProperties.get(LoadStmt.TIMEZONE); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadLoadingTask.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadLoadingTask.java index 450972900f1e1db..9a2fdb647f4d021 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadLoadingTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadLoadingTask.java @@ -61,6 +61,7 @@ public class LoadLoadingTask extends LoadTask { private final long jobDeadlineMs; private final long execMemLimit; private final boolean strictMode; + private final boolean isPartialUpdate; private final long txnId; private final String timezone; // timeout of load job, in seconds @@ -78,7 +79,7 @@ public class LoadLoadingTask extends LoadTask { public LoadLoadingTask(Database db, OlapTable table, BrokerDesc brokerDesc, List fileGroups, - long jobDeadlineMs, long execMemLimit, boolean strictMode, + long jobDeadlineMs, long execMemLimit, boolean strictMode, boolean isPartialUpdate, long txnId, LoadTaskCallback callback, String timezone, long timeoutS, int loadParallelism, int sendBatchParallelism, boolean loadZeroTolerance, Profile jobProfile, boolean singleTabletLoadPerSink, @@ -91,6 +92,7 @@ public LoadLoadingTask(Database db, OlapTable table, this.jobDeadlineMs = jobDeadlineMs; this.execMemLimit = execMemLimit; this.strictMode = strictMode; + this.isPartialUpdate = isPartialUpdate; this.txnId = txnId; this.failMsg = new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL); this.retryTime = 2; // 2 times is enough @@ -108,7 +110,7 @@ public void init(TUniqueId loadId, List> fileStatusList, int fileNum, UserIdentity userInfo) throws UserException { this.loadId = loadId; planner = new LoadingTaskPlanner(callback.getCallbackId(), txnId, db.getId(), table, brokerDesc, fileGroups, - strictMode, timezone, this.timeoutS, this.loadParallelism, this.sendBatchParallelism, + strictMode, isPartialUpdate, timezone, this.timeoutS, this.loadParallelism, this.sendBatchParallelism, this.useNewLoadScanNode, userInfo); planner.plan(loadId, fileStatusList, fileNum); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java index 14b2c319903f993..17ee58bd639ec1d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java @@ -51,6 +51,7 @@ import org.apache.logging.log4j.Logger; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Set; @@ -65,6 +66,7 @@ public class LoadingTaskPlanner { private final BrokerDesc brokerDesc; private final List fileGroups; private final boolean strictMode; + private final boolean isPartialUpdate; private final long timeoutS; // timeout of load job, in second private final int loadParallelism; private final int sendBatchParallelism; @@ -83,7 +85,7 @@ public class LoadingTaskPlanner { public LoadingTaskPlanner(Long loadJobId, long txnId, long dbId, OlapTable table, BrokerDesc brokerDesc, List brokerFileGroups, - boolean strictMode, String timezone, long timeoutS, int loadParallelism, + boolean strictMode, boolean isPartialUpdate, String timezone, long timeoutS, int loadParallelism, int sendBatchParallelism, boolean useNewLoadScanNode, UserIdentity userInfo) { this.loadJobId = loadJobId; this.txnId = txnId; @@ -92,6 +94,7 @@ public LoadingTaskPlanner(Long loadJobId, long txnId, long dbId, OlapTable table this.brokerDesc = brokerDesc; this.fileGroups = brokerFileGroups; this.strictMode = strictMode; + this.isPartialUpdate = isPartialUpdate; this.analyzer.setTimezone(timezone); this.timeoutS = timeoutS; this.loadParallelism = loadParallelism; @@ -113,8 +116,37 @@ public void plan(TUniqueId loadId, List> fileStatusesLis TupleDescriptor destTupleDesc = descTable.createTupleDescriptor(); TupleDescriptor scanTupleDesc = destTupleDesc; scanTupleDesc = descTable.createTupleDescriptor("ScanTuple"); + if (isPartialUpdate && !table.getEnableUniqueKeyMergeOnWrite()) { + throw new UserException("Only unique key merge on write support partial update"); + } + + HashSet partialUpdateInputColumns = new HashSet<>(); + if (isPartialUpdate) { + for (Column col : table.getFullSchema()) { + boolean existInExpr = false; + for (ImportColumnDesc importColumnDesc : fileGroups.get(0).getColumnExprList()) { + if (importColumnDesc.getColumnName() != null + && importColumnDesc.getColumnName().equals(col.getName())) { + if (!col.isVisible() && !Column.DELETE_SIGN.equals(col.getName())) { + throw new UserException("Partial update should not include invisible column except" + + " delete sign column: " + col.getName()); + } + partialUpdateInputColumns.add(col.getName()); + existInExpr = true; + break; + } + } + if (col.isKey() && !existInExpr) { + throw new UserException("Partial update should include all key columns, missing: " + col.getName()); + } + } + } + // use full schema to fill the descriptor table for (Column col : table.getFullSchema()) { + if (isPartialUpdate && !partialUpdateInputColumns.contains(col.getName())) { + continue; + } SlotDescriptor slotDesc = descTable.addSlotDescriptor(destTupleDesc); slotDesc.setIsMaterialized(true); slotDesc.setColumn(col); @@ -123,6 +155,13 @@ public void plan(TUniqueId loadId, List> fileStatusesLis scanSlotDesc.setIsMaterialized(true); scanSlotDesc.setColumn(col); scanSlotDesc.setIsNullable(col.isAllowNull()); + scanSlotDesc.setAutoInc(col.isAutoInc()); + if (col.isAutoInc()) { + // auto-increment column should be non-nullable + // however, here we use `NullLiteral` to indicate that a cell should + // be filled with generated value in `VOlapTableSink::_fill_auto_inc_cols()` + scanSlotDesc.setIsNullable(true); + } if (fileGroups.size() > 0) { for (ImportColumnDesc importColumnDesc : fileGroups.get(0).getColumnExprList()) { try { @@ -155,6 +194,7 @@ public void plan(TUniqueId loadId, List> fileStatusesLis OlapTableSink olapTableSink = new OlapTableSink(table, destTupleDesc, partitionIds, Config.enable_single_replica_load); olapTableSink.init(loadId, txnId, dbId, timeoutS, sendBatchParallelism, false, strictMode); + olapTableSink.setPartialUpdateInputColumns(isPartialUpdate, partialUpdateInputColumns); olapTableSink.complete(analyzer); // 3. Plan fragment diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/AccessControllerManager.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/AccessControllerManager.java index 3e0ff23915b50fd..257a6e88bc5a4d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/AccessControllerManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/AccessControllerManager.java @@ -59,7 +59,7 @@ public AccessControllerManager(Auth auth) { ctlToCtlAccessController.put(InternalCatalog.INTERNAL_CATALOG_NAME, internalAccessController); } - private CatalogAccessController getAccessControllerOrDefault(String ctl) { + public CatalogAccessController getAccessControllerOrDefault(String ctl) { CatalogAccessController catalogAccessController = ctlToCtlAccessController.get(ctl); if (catalogAccessController != null) { return catalogAccessController; @@ -77,7 +77,7 @@ private synchronized void lazyLoadCtlAccessController(ExternalCatalog catalog) { if (ctlToCtlAccessController.containsKey(catalog.getName())) { return; } - catalog.initAccessController(); + catalog.initAccessController(false); if (!ctlToCtlAccessController.containsKey(catalog.getName())) { ctlToCtlAccessController.put(catalog.getName(), internalAccessController); } @@ -88,14 +88,17 @@ public boolean checkIfAccessControllerExist(String ctl) { return ctlToCtlAccessController.containsKey(ctl); } - public void createAccessController(String ctl, String acFactoryClassName, Map prop) { + public void createAccessController(String ctl, String acFactoryClassName, Map prop, + boolean isDryRun) { Class factoryClazz = null; try { factoryClazz = Class.forName(acFactoryClassName); AccessControllerFactory factory = (AccessControllerFactory) factoryClazz.newInstance(); CatalogAccessController accessController = factory.createAccessController(prop); - ctlToCtlAccessController.put(ctl, accessController); - LOG.info("create access controller {} for catalog {}", ctl, acFactoryClassName); + if (!isDryRun) { + ctlToCtlAccessController.put(ctl, accessController); + LOG.info("create access controller {} for catalog {}", ctl, acFactoryClassName); + } } catch (ClassNotFoundException e) { throw new RuntimeException(e); } catch (InstantiationException e) { @@ -130,7 +133,10 @@ public boolean checkCtlPriv(ConnectContext ctx, String ctl, PrivPredicate wanted public boolean checkCtlPriv(UserIdentity currentUser, String ctl, PrivPredicate wanted) { boolean hasGlobal = sysAccessController.checkGlobalPriv(currentUser, wanted); - return getAccessControllerOrDefault(ctl).checkCtlPriv(hasGlobal, currentUser, ctl, wanted); + // for checking catalog priv, always use InternalCatalogAccessController. + // because catalog priv is only saved in InternalCatalogAccessController. + return getAccessControllerOrDefault(InternalCatalog.INTERNAL_CATALOG_NAME).checkCtlPriv(hasGlobal, currentUser, + ctl, wanted); } // ==== Database ==== diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index bb1ed834687a83d..6dc75f2632f8b08 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -29,6 +29,7 @@ import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.glue.translator.PhysicalPlanTranslator; import org.apache.doris.nereids.glue.translator.PlanTranslatorContext; +import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.jobs.executor.Optimizer; import org.apache.doris.nereids.jobs.executor.Rewriter; import org.apache.doris.nereids.memo.Group; @@ -69,6 +70,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -347,26 +349,61 @@ private ScheduledExecutorService runTimeoutExecutor() { return executor; } + /** + * getting hints explain string, which specified by enumerate and show in lists + * @param hintMap hint map recorded in statement context + * @return explain string shows using of hint + */ + public String getHintExplainString(Map hintMap) { + String used = ""; + String unUsed = ""; + String syntaxError = ""; + for (Map.Entry entry : hintMap.entrySet()) { + switch (entry.getValue().getStatus()) { + case UNUSED: + unUsed = unUsed + " " + entry.getValue().getExplainString(); + break; + case SYNTAX_ERROR: + syntaxError = syntaxError + " " + entry.getValue().getExplainString() + + " Msg:" + entry.getValue().getErrorMessage(); + break; + case SUCCESS: + used = used + " " + entry.getValue().getExplainString(); + break; + default: + break; + } + } + return "\nUsed:" + used + "\nUnUsed:" + unUsed + "\nSyntaxError:" + syntaxError; + } + @Override public String getExplainString(ExplainOptions explainOptions) { ExplainLevel explainLevel = getExplainLevel(explainOptions); + String plan = ""; switch (explainLevel) { case PARSED_PLAN: - return parsedPlan.treeString(); + plan = parsedPlan.treeString(); + break; case ANALYZED_PLAN: - return analyzedPlan.treeString(); + plan = analyzedPlan.treeString(); + break; case REWRITTEN_PLAN: - return rewrittenPlan.treeString(); + plan = rewrittenPlan.treeString(); + break; case OPTIMIZED_PLAN: - return "cost = " + cost + "\n" + optimizedPlan.treeString(); + plan = "cost = " + cost + "\n" + optimizedPlan.treeString(); + break; case SHAPE_PLAN: - return optimizedPlan.shape(""); + plan = optimizedPlan.shape(""); + break; case MEMO_PLAN: - return cascadesContext.getMemo().toString() + plan = cascadesContext.getMemo().toString() + "\n\n========== OPTIMIZED PLAN ==========\n" + optimizedPlan.treeString(); + break; case ALL_PLAN: - return "========== PARSED PLAN ==========\n" + plan = "========== PARSED PLAN ==========\n" + parsedPlan.treeString() + "\n\n" + "========== ANALYZED PLAN ==========\n" + analyzedPlan.treeString() + "\n\n" @@ -374,9 +411,15 @@ public String getExplainString(ExplainOptions explainOptions) { + rewrittenPlan.treeString() + "\n\n" + "========== OPTIMIZED PLAN ==========\n" + optimizedPlan.treeString(); + break; default: - return super.getExplainString(explainOptions); + plan = super.getExplainString(explainOptions); + } + if (!statementContext.getHintMap().isEmpty()) { + String hint = getHintExplainString(cascadesContext.getStatementContext().getHintMap()); + return plan + hint; } + return plan; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/PlanContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/PlanContext.java index 11b0f718346a64f..847d18afdc81ed9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/PlanContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/PlanContext.java @@ -22,7 +22,6 @@ import org.apache.doris.statistics.Statistics; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; /** @@ -33,7 +32,7 @@ */ public class PlanContext { - private List childrenStats = new ArrayList<>(); + private final List childrenStats; private Statistics planStats; private final int arity; private boolean isBroadcastJoin = false; @@ -43,19 +42,16 @@ public class PlanContext { */ public PlanContext(GroupExpression groupExpression) { this.arity = groupExpression.arity(); - if (groupExpression.getOwnerGroup() == null) { - return; - } - planStats = groupExpression.getOwnerGroup().getStatistics(); - childrenStats = new ArrayList<>(groupExpression.arity()); + this.planStats = groupExpression.getOwnerGroup().getStatistics(); + this.childrenStats = new ArrayList<>(groupExpression.arity()); for (int i = 0; i < groupExpression.arity(); i++) { childrenStats.add(groupExpression.childStatistics(i)); } } - public PlanContext(Statistics planStats, Statistics... childrenStats) { + public PlanContext(Statistics planStats, List childrenStats) { this.planStats = planStats; - this.childrenStats = Arrays.asList(childrenStats); + this.childrenStats = childrenStats; this.arity = this.childrenStats.size(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 5f7dcbe0bd7c945..ceb2e22721b19eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.View; import org.apache.doris.common.IdGenerator; import org.apache.doris.common.Pair; +import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.rules.analysis.ColumnAliasGenerator; import org.apache.doris.nereids.trees.expressions.CTEId; @@ -72,6 +73,8 @@ public class StatementContext { private boolean isDpHyp = false; private boolean isOtherJoinReorder = false; + private boolean isLeadingJoin = false; + private final IdGenerator exprIdGenerator = ExprId.createGenerator(); private final IdGenerator objectIdGenerator = ObjectId.createGenerator(); private final IdGenerator relationIdGenerator = RelationId.createGenerator(); @@ -85,6 +88,7 @@ public class StatementContext { private final Map, Group>>> cteIdToConsumerGroup = new HashMap<>(); private final Map rewrittenCtePlan = new HashMap<>(); private final Set views = Sets.newHashSet(); + private final Map hintMap = Maps.newLinkedHashMap(); public StatementContext() { this.connectContext = ConnectContext.get(); @@ -143,6 +147,14 @@ public void setDpHyp(boolean dpHyp) { isDpHyp = dpHyp; } + public boolean isLeadingJoin() { + return isLeadingJoin; + } + + public void setLeadingJoin(boolean leadingJoin) { + isLeadingJoin = leadingJoin; + } + public boolean isOtherJoinReorder() { return isOtherJoinReorder; } @@ -181,6 +193,10 @@ public synchronized T getOrRegisterCache(String key, Supplier cacheSuppli return supplier.get(); } + public Map getHintMap() { + return hintMap; + } + public ColumnAliasGenerator getColumnAliasGenerator() { return columnAliasGenerator == null ? columnAliasGenerator = new ColumnAliasGenerator() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/CostModelV1.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/CostModelV1.java index 1b1c72f37a2453c..c802148fe0e202d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/CostModelV1.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/cost/CostModelV1.java @@ -149,14 +149,14 @@ public Cost visitPhysicalQuickSort( // TODO: consider two-phase sort and enforcer. Statistics statistics = context.getStatisticsWithCheck(); Statistics childStatistics = context.getChildStatistics(0); + + double childRowCount = childStatistics.getRowCount(); + double rowCount = statistics.getRowCount(); if (physicalQuickSort.getSortPhase().isGather()) { // Now we do more like two-phase sort, so penalise one-phase sort - statistics = statistics.withRowCount(statistics.getRowCount() * 100); + rowCount *= 100; } - return CostV1.of( - childStatistics.getRowCount(), - statistics.getRowCount(), - childStatistics.getRowCount()); + return CostV1.of(childRowCount, rowCount, childRowCount); } @Override @@ -164,14 +164,14 @@ public Cost visitPhysicalTopN(PhysicalTopN topN, PlanContext con // TODO: consider two-phase sort and enforcer. Statistics statistics = context.getStatisticsWithCheck(); Statistics childStatistics = context.getChildStatistics(0); + + double childRowCount = childStatistics.getRowCount(); + double rowCount = statistics.getRowCount(); if (topN.getSortPhase().isGather()) { // Now we do more like two-phase sort, so penalise one-phase sort - statistics = statistics.withRowCount(statistics.getRowCount() * 100); + rowCount *= 100; } - return CostV1.of( - childStatistics.getRowCount(), - statistics.getRowCount(), - childStatistics.getRowCount()); + return CostV1.of(childRowCount, rowCount, childRowCount); } @Override 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 e16e851fe324da3..c8b4a701a3b9bbf 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 @@ -91,7 +91,6 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEAnchor; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEConsumer; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEProducer; -import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalDeferMaterializeOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalDeferMaterializeResultSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalDeferMaterializeTopN; @@ -437,7 +436,7 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla } scanNode.addConjuncts(translateToLegacyConjuncts(fileScan.getConjuncts())); - scanNode.setPushDownAggNoGrouping(context.getTablePushAggOp(table.getId())); + scanNode.setPushDownAggNoGrouping(context.getRelationPushAggOp(fileScan.getRelationId())); TableName tableName = new TableName(null, "", ""); TableRef ref = new TableRef(tableName, null, null); @@ -576,6 +575,7 @@ public PlanFragment visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanTransla expr, olapScanNode, context) ) ); + olapScanNode.setPushDownAggNoGrouping(context.getRelationPushAggOp(olapScan.getRelationId())); // TODO: we need to remove all finalizeForNereids olapScanNode.finalizeForNereids(); // Create PlanFragment @@ -817,8 +817,8 @@ public PlanFragment visitPhysicalStorageLayerAggregate( + storageLayerAggregate.getAggOp()); } - context.setTablePushAggOp( - ((PhysicalCatalogRelation) storageLayerAggregate.getRelation()).getTable().getId(), pushAggOp); + context.setRelationPushAggOp( + storageLayerAggregate.getRelation().getRelationId(), pushAggOp); PlanFragment planFragment = storageLayerAggregate.getRelation().accept(this, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java index 4e90d3eea251ae4..4136def70179886 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java @@ -32,6 +32,7 @@ import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.VirtualSlotReference; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEConsumer; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEProducer; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; @@ -94,7 +95,7 @@ public class PlanTranslatorContext { private final Map cteScanNodeMap = Maps.newHashMap(); - private final Map tablePushAggOp = Maps.newHashMap(); + private final Map tablePushAggOp = Maps.newHashMap(); public PlanTranslatorContext(CascadesContext ctx) { this.translator = new RuntimeFilterTranslator(ctx.getRuntimeFilterContext()); @@ -239,11 +240,11 @@ public DescriptorTable getDescTable() { return descTable; } - public void setTablePushAggOp(Long tableId, TPushAggOp aggOp) { - tablePushAggOp.put(tableId, aggOp); + public void setRelationPushAggOp(RelationId relationId, TPushAggOp aggOp) { + tablePushAggOp.put(relationId, aggOp); } - public TPushAggOp getTablePushAggOp(Long tableId) { - return tablePushAggOp.getOrDefault(tableId, TPushAggOp.NONE); + public TPushAggOp getRelationPushAggOp(RelationId relationId) { + return tablePushAggOp.getOrDefault(relationId, TPushAggOp.NONE); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/Hint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/Hint.java new file mode 100644 index 000000000000000..8640ab3f5b243f3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/Hint.java @@ -0,0 +1,85 @@ +// 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.hint; + +import java.util.Objects; + +/** + * select hint. + * e.g. set_var(query_timeout='1800', exec_mem_limit='2147483648') + */ +public class Hint { + // e.g. set_var + private String hintName; + + private HintStatus status; + + private String errorMessage = ""; + + /** + * hint status which need to show in explain when it is not used or have syntax error + */ + public enum HintStatus { + UNUSED, + SYNTAX_ERROR, + SUCCESS + } + + public Hint(String hintName) { + this.hintName = Objects.requireNonNull(hintName, "hintName can not be null"); + this.status = HintStatus.UNUSED; + } + + public void setHintName(String hintName) { + this.hintName = hintName; + } + + public void setStatus(HintStatus status) { + this.status = status; + } + + public HintStatus getStatus() { + return status; + } + + public boolean isSuccess() { + return getStatus() == HintStatus.SUCCESS; + } + + public boolean isSyntaxError() { + return getStatus() == HintStatus.SYNTAX_ERROR; + } + + public String getHintName() { + return hintName; + } + + public String getErrorMessage() { + return errorMessage; + } + + public void setErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + } + + public String getExplainString() { + StringBuilder out = new StringBuilder(); + out.append("\nHint:\n"); + return out.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/JoinConstraint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/JoinConstraint.java new file mode 100644 index 000000000000000..dfe26d9945c4c01 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/JoinConstraint.java @@ -0,0 +1,88 @@ +// 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.hint; + +import org.apache.doris.nereids.trees.plans.JoinType; + +/** + * Join constraint which helps for leading to construct outer join , semi join and anti join + */ +public class JoinConstraint { + private final Long minLeftHand; + private final Long minRightHand; + + private final Long leftHand; + private final Long rightHand; + + private final JoinType joinType; + + private final boolean lhsStrict; + + private boolean isReversed; + + /** + * join constraints which means restriction to some logical not equivalence of left join and semi join + * @param minLeftHand minimal left hand table bitmap which needed for a special join + * @param minRightHand minimal right hand table bitmap which needed for a special join + * @param leftHand left hand table bitmap below current join + * @param rightHand right hand table bitmap below current join + * @param joinType join type, here we only have full outer join, left join, semi join and anti join + * @param lhsStrict is left hand side strict + */ + public JoinConstraint(Long minLeftHand, Long minRightHand, Long leftHand, Long rightHand, + JoinType joinType, boolean lhsStrict) { + this.minLeftHand = minLeftHand; + this.minRightHand = minRightHand; + this.leftHand = leftHand; + this.rightHand = rightHand; + this.joinType = joinType; + this.lhsStrict = lhsStrict; + } + + public JoinType getJoinType() { + return joinType; + } + + public Long getLeftHand() { + return leftHand; + } + + public Long getRightHand() { + return rightHand; + } + + public Long getMinLeftHand() { + return minLeftHand; + } + + public Long getMinRightHand() { + return minRightHand; + } + + public boolean isLhsStrict() { + return lhsStrict; + } + + public void setReversed(boolean reversed) { + isReversed = reversed; + } + + public boolean isReversed() { + return isReversed; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/LeadingHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/LeadingHint.java new file mode 100644 index 000000000000000..af5c4f500fc2d0a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/LeadingHint.java @@ -0,0 +1,503 @@ +// 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.hint; + +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.JoinHint; +import org.apache.doris.nereids.trees.plans.JoinType; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.RelationId; +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.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; +import org.apache.doris.nereids.util.JoinUtils; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.Stack; + +/** + * select hint. + * e.g. set_var(query_timeout='1800', exec_mem_limit='2147483648') + */ +public class LeadingHint extends Hint { + private String originalString = ""; + private final List tablelist = new ArrayList<>(); + private final List levellist = new ArrayList<>(); + + private final Map relationIdToScanMap = Maps.newLinkedHashMap(); + + private final List> relationIdAndTableName = new ArrayList<>(); + + private final Map exprIdToTableNameMap = Maps.newLinkedHashMap(); + + private final List> filters = new ArrayList<>(); + + private final List joinConstraintList = new ArrayList<>(); + + private Long innerJoinBitmap = 0L; + + public LeadingHint(String hintName) { + super(hintName); + } + + /** + * Leading hint data structure before using + * @param hintName Leading + * @param parameters table name mixed with left and right brace + */ + public LeadingHint(String hintName, List parameters, String originalString) { + super(hintName); + this.originalString = originalString; + int level = 0; + for (String parameter : parameters) { + if (parameter.equals("{")) { + ++level; + } else if (parameter.equals("}")) { + level--; + } else { + tablelist.add(parameter); + levellist.add(level); + } + } + } + + public List getTablelist() { + return tablelist; + } + + public List getLevellist() { + return levellist; + } + + public Map getRelationIdToScanMap() { + return relationIdToScanMap; + } + + @Override + public String getExplainString() { + StringBuilder out = new StringBuilder(); + out.append(originalString); + return out.toString(); + } + + /** + * Get logical plan by table name recorded in leading hint. if can not get, means leading has syntax error + * or need to update. So return null should be deal with when call + * @param name table name + * @return logical plan recorded when binding + */ + public LogicalPlan getLogicalPlanByName(String name) { + RelationId id = findRelationIdAndTableName(name); + if (id == null) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("can not find table: " + name); + return null; + } + return relationIdToScanMap.get(id); + } + + /** + * putting pair into list, if relation id already exist update table name + * @param relationIdTableNamePair pair of relation id and table name to be inserted + */ + public void putRelationIdAndTableName(Pair relationIdTableNamePair) { + boolean isUpdate = false; + for (Pair pair : relationIdAndTableName) { + if (pair.first.equals(relationIdTableNamePair.first)) { + pair.second = relationIdTableNamePair.second; + isUpdate = true; + } + } + if (!isUpdate) { + relationIdAndTableName.add(relationIdTableNamePair); + } + } + + /** + * putting pair into list, if relation id already exist update table name + * @param relationIdTableNamePair pair of relation id and table name to be inserted + */ + public void updateRelationIdByTableName(Pair relationIdTableNamePair) { + boolean isUpdate = false; + for (Pair pair : relationIdAndTableName) { + if (pair.second.equals(relationIdTableNamePair.second)) { + pair.first = relationIdTableNamePair.first; + isUpdate = true; + } + } + if (!isUpdate) { + relationIdAndTableName.add(relationIdTableNamePair); + } + } + + /** + * find relation id and table name pair, relation id is unique, but table name is not + * @param name table name + * @return relation id + */ + public RelationId findRelationIdAndTableName(String name) { + for (Pair pair : relationIdAndTableName) { + if (pair.second.equals(name)) { + return pair.first; + } + } + return null; + } + + private boolean hasSameName() { + Set tableSet = Sets.newHashSet(); + for (String table : tablelist) { + if (!tableSet.add(table)) { + return true; + } + } + return false; + } + + public Map getExprIdToTableNameMap() { + return exprIdToTableNameMap; + } + + public List> getFilters() { + return filters; + } + + public List getJoinConstraintList() { + return joinConstraintList; + } + + public Long getInnerJoinBitmap() { + return innerJoinBitmap; + } + + public void setInnerJoinBitmap(Long innerJoinBitmap) { + this.innerJoinBitmap = innerJoinBitmap; + } + + /** + * try to get join constraint, if can not get, it means join is inner join, + * @param joinTableBitmap table bitmap below this join + * @param leftTableBitmap table bitmap below right child + * @param rightTableBitmap table bitmap below right child + * @return boolean value used for judging whether the join is legal, and should this join need to reverse + */ + public Pair getJoinConstraint(Long joinTableBitmap, Long leftTableBitmap, + Long rightTableBitmap) { + boolean reversed = false; + boolean mustBeLeftjoin = false; + + JoinConstraint matchedJoinConstraint = null; + + for (JoinConstraint joinConstraint : joinConstraintList) { + if (!LongBitmap.isOverlap(joinConstraint.getMinRightHand(), joinTableBitmap)) { + continue; + } + + if (LongBitmap.isSubset(joinTableBitmap, joinConstraint.getMinRightHand())) { + continue; + } + + if (LongBitmap.isSubset(joinConstraint.getMinLeftHand(), leftTableBitmap) + && LongBitmap.isSubset(joinConstraint.getMinRightHand(), leftTableBitmap)) { + continue; + } + + if (LongBitmap.isSubset(joinConstraint.getMinLeftHand(), rightTableBitmap) + && LongBitmap.isSubset(joinConstraint.getMinRightHand(), rightTableBitmap)) { + continue; + } + + if (joinConstraint.getJoinType().isSemiJoin()) { + if (LongBitmap.isSubset(joinConstraint.getRightHand(), leftTableBitmap) + && !LongBitmap.isSubset(joinConstraint.getRightHand(), leftTableBitmap)) { + continue; + } + if (LongBitmap.isSubset(joinConstraint.getRightHand(), rightTableBitmap) + && !joinConstraint.getRightHand().equals(rightTableBitmap)) { + continue; + } + } + + if (LongBitmap.isSubset(joinConstraint.getMinLeftHand(), leftTableBitmap) + && LongBitmap.isSubset(joinConstraint.getMinRightHand(), rightTableBitmap)) { + if (matchedJoinConstraint != null) { + return Pair.of(null, false); + } + matchedJoinConstraint = joinConstraint; + reversed = false; + } else if (LongBitmap.isSubset(joinConstraint.getMinLeftHand(), rightTableBitmap) + && LongBitmap.isSubset(joinConstraint.getMinRightHand(), leftTableBitmap)) { + if (matchedJoinConstraint != null) { + return Pair.of(null, false); + } + matchedJoinConstraint = joinConstraint; + reversed = true; + } else if (joinConstraint.getJoinType().isSemiJoin() + && joinConstraint.getRightHand().equals(rightTableBitmap)) { + if (matchedJoinConstraint != null) { + return Pair.of(null, false); + } + matchedJoinConstraint = joinConstraint; + reversed = false; + } else if (joinConstraint.getJoinType().isSemiJoin() + && joinConstraint.getRightHand().equals(leftTableBitmap)) { + /* Reversed semijoin case */ + if (matchedJoinConstraint != null) { + return Pair.of(null, false); + } + matchedJoinConstraint = joinConstraint; + reversed = true; + } else { + if (LongBitmap.isOverlap(leftTableBitmap, joinConstraint.getMinRightHand()) + && LongBitmap.isOverlap(rightTableBitmap, joinConstraint.getMinRightHand())) { + continue; + } + + if (!joinConstraint.getJoinType().isLeftJoin() + || LongBitmap.isOverlap(joinTableBitmap, joinConstraint.getMinLeftHand())) { + return Pair.of(null, false); + } + + mustBeLeftjoin = true; + } + } + if (mustBeLeftjoin && (matchedJoinConstraint == null || !matchedJoinConstraint.getJoinType().isLeftJoin() + || !matchedJoinConstraint.isLhsStrict())) { + return Pair.of(null, false); + } + // this means inner join + if (matchedJoinConstraint == null) { + return Pair.of(null, true); + } + matchedJoinConstraint.setReversed(reversed); + return Pair.of(matchedJoinConstraint, true); + } + + /** + * Try to get join type of two random logical scan or join node table bitmap + * @param left left side table bitmap + * @param right right side table bitmap + * @return join type or failure + */ + public JoinType computeJoinType(Long left, Long right, List conditions) { + Pair joinConstraintBooleanPair + = getJoinConstraint(LongBitmap.or(left, right), left, right); + if (!joinConstraintBooleanPair.second) { + this.setStatus(HintStatus.UNUSED); + } else if (joinConstraintBooleanPair.first == null) { + if (conditions.isEmpty()) { + return JoinType.CROSS_JOIN; + } + return JoinType.INNER_JOIN; + } else { + JoinConstraint joinConstraint = joinConstraintBooleanPair.first; + if (joinConstraint.isReversed()) { + return joinConstraint.getJoinType().swap(); + } else { + return joinConstraint.getJoinType(); + } + } + if (conditions.isEmpty()) { + return JoinType.CROSS_JOIN; + } + return JoinType.INNER_JOIN; + } + + /** + * using leading to generate plan, it could be failed, if failed set leading status to unused or syntax error + * @return plan + */ + public Plan generateLeadingJoinPlan() { + this.setStatus(HintStatus.SUCCESS); + Stack> stack = new Stack<>(); + int index = 0; + LogicalPlan logicalPlan = getLogicalPlanByName(getTablelist().get(index)); + if (logicalPlan == null) { + return null; + } + logicalPlan = makeFilterPlanIfExist(getFilters(), logicalPlan); + assert (logicalPlan != null); + stack.push(Pair.of(getLevellist().get(index), logicalPlan)); + int stackTopLevel = getLevellist().get(index++); + while (index < getTablelist().size()) { + int currentLevel = getLevellist().get(index); + if (currentLevel == stackTopLevel) { + // should return error if can not found table + logicalPlan = getLogicalPlanByName(getTablelist().get(index++)); + if (logicalPlan == null) { + return null; + } + logicalPlan = makeFilterPlanIfExist(getFilters(), logicalPlan); + Pair newStackTop = stack.peek(); + while (!(stack.isEmpty() || stackTopLevel != newStackTop.first)) { + // check join is legal and get join type + newStackTop = stack.pop(); + List conditions = getJoinConditions( + getFilters(), newStackTop.second, logicalPlan); + Pair, List> pair = JoinUtils.extractExpressionForHashTable( + newStackTop.second.getOutput(), logicalPlan.getOutput(), conditions); + JoinType joinType = computeJoinType(getBitmap(newStackTop.second), + getBitmap(logicalPlan), conditions); + if (!this.isSuccess()) { + return null; + } + // get joinType + LogicalJoin logicalJoin = new LogicalJoin<>(joinType, pair.first, + pair.second, + JoinHint.NONE, + Optional.empty(), + newStackTop.second, + logicalPlan); + logicalJoin.setBitmap(LongBitmap.or(getBitmap(newStackTop.second), getBitmap(logicalPlan))); + if (stackTopLevel > 0) { + stackTopLevel--; + } + if (!stack.isEmpty()) { + newStackTop = stack.peek(); + } + logicalPlan = logicalJoin; + } + stack.push(Pair.of(stackTopLevel, logicalPlan)); + } else { + // push + logicalPlan = getLogicalPlanByName(getTablelist().get(index++)); + if (logicalPlan == null) { + return null; + } + logicalPlan = makeFilterPlanIfExist(getFilters(), logicalPlan); + stack.push(Pair.of(currentLevel, logicalPlan)); + stackTopLevel = currentLevel; + } + } + + LogicalJoin finalJoin = (LogicalJoin) stack.pop().second; + // we want all filters been remove + if (!getFilters().isEmpty()) { + List conditions = getLastConditions(getFilters()); + Pair, List> pair = JoinUtils.extractExpressionForHashTable( + finalJoin.left().getOutput(), finalJoin.right().getOutput(), conditions); + finalJoin = new LogicalJoin<>(finalJoin.getJoinType(), pair.first, + pair.second, + JoinHint.NONE, + Optional.empty(), + finalJoin.left(), + finalJoin.right()); + } + if (finalJoin != null) { + this.setStatus(HintStatus.SUCCESS); + } + return finalJoin; + } + + private List getJoinConditions(List> filters, + LogicalPlan left, LogicalPlan right) { + List joinConditions = new ArrayList<>(); + for (int i = filters.size() - 1; i >= 0; i--) { + Pair filterPair = filters.get(i); + Long tablesBitMap = LongBitmap.or(getBitmap(left), getBitmap(right)); + // left one is smaller set + if (LongBitmap.isSubset(filterPair.first, tablesBitMap)) { + joinConditions.add(filterPair.second); + filters.remove(i); + } + } + return joinConditions; + } + + private List getLastConditions(List> filters) { + List joinConditions = new ArrayList<>(); + for (int i = filters.size() - 1; i >= 0; i--) { + Pair filterPair = filters.get(i); + joinConditions.add(filterPair.second); + filters.remove(i); + } + return joinConditions; + } + + private LogicalPlan makeFilterPlanIfExist(List> filters, LogicalPlan scan) { + Set newConjuncts = new HashSet<>(); + for (int i = filters.size() - 1; i >= 0; i--) { + Pair filterPair = filters.get(i); + if (LongBitmap.isSubset(filterPair.first, getBitmap(scan))) { + newConjuncts.add(filterPair.second); + filters.remove(i); + } + } + if (newConjuncts.isEmpty()) { + return scan; + } else { + return new LogicalFilter<>(newConjuncts, scan); + } + } + + private Long getBitmap(LogicalPlan root) { + if (root instanceof LogicalJoin) { + return ((LogicalJoin) root).getBitmap(); + } else if (root instanceof LogicalRelation) { + return LongBitmap.set(0L, (((LogicalRelation) root).getRelationId().asInt())); + } else if (root instanceof LogicalFilter) { + return getBitmap((LogicalPlan) root.child(0)); + } else if (root instanceof LogicalProject) { + return getBitmap((LogicalPlan) root.child(0)); + } else { + return null; + } + } + + /** + * get leading containing tables which means leading wants to combine tables into joins + * @return long value represent tables we included + */ + public Long getLeadingTableBitmap(List tables) { + Long totalBitmap = 0L; + if (hasSameName()) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("duplicated table"); + return totalBitmap; + } + if (getTablelist().size() != tables.size()) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("tables should be same as join tables"); + return totalBitmap; + } + for (int index = 0; index < getTablelist().size(); index++) { + RelationId id = findRelationIdAndTableName(getTablelist().get(index)); + if (id == null) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("can not find table: " + getTablelist().get(index)); + return totalBitmap; + } + totalBitmap = LongBitmap.set(totalBitmap, id.asInt()); + } + return totalBitmap; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java index a620d6b5b13b6c9..ed67b44f1fd619f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java @@ -19,6 +19,8 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.jobs.rewrite.RewriteJob; +import org.apache.doris.nereids.processor.pre.EliminateLogicalSelectHint; +import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.analysis.AdjustAggregateNullableForEmptySet; import org.apache.doris.nereids.rules.analysis.AnalyzeCTE; import org.apache.doris.nereids.rules.analysis.BindExpression; @@ -28,7 +30,9 @@ import org.apache.doris.nereids.rules.analysis.CheckAnalysis; import org.apache.doris.nereids.rules.analysis.CheckBound; import org.apache.doris.nereids.rules.analysis.CheckPolicy; +import org.apache.doris.nereids.rules.analysis.EliminateGroupByConstant; import org.apache.doris.nereids.rules.analysis.FillUpMissingSlots; +import org.apache.doris.nereids.rules.analysis.NormalizeAggregate; import org.apache.doris.nereids.rules.analysis.NormalizeRepeat; import org.apache.doris.nereids.rules.analysis.ProjectToGlobalAggregate; import org.apache.doris.nereids.rules.analysis.ProjectWithDistinctToAggregate; @@ -79,6 +83,8 @@ public void analyze() { private static List buildAnalyzeJobs(Optional customTableResolver) { return jobs( + // we should eliminate hint after "Subquery unnesting" because some hint maybe exist in the CTE or subquery. + custom(RuleType.ELIMINATE_HINT, EliminateLogicalSelectHint::new), topDown(new AnalyzeCTE()), bottomUp( new BindRelation(customTableResolver), @@ -106,9 +112,14 @@ private static List buildAnalyzeJobs(Optional c // LogicalProject for normalize. This rule depends on FillUpMissingSlots to fill up slots. new NormalizeRepeat() ), - bottomUp(new SubqueryToApply()), bottomUp(new AdjustAggregateNullableForEmptySet()), - bottomUp(new CheckAnalysis()) + // run CheckAnalysis before EliminateGroupByConstant in order to report error message correctly like bellow + // select SUM(lo_tax) FROM lineorder group by 1; + // errCode = 2, detailMessage = GROUP BY expression must not contain aggregate functions: sum(lo_tax) + bottomUp(new CheckAnalysis()), + topDown(new EliminateGroupByConstant()), + topDown(new NormalizeAggregate()), + bottomUp(new SubqueryToApply()) ); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java index b2bd5e63ec209f6..432d2dcd2242664 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Optimizer.java @@ -68,7 +68,6 @@ public void execute() { cascadesContext.getJobScheduler().executeJobPool(cascadesContext); } - // DependsRules: EnsureProjectOnTopJoin.class private void dpHypOptimize() { Group root = cascadesContext.getMemo().getRoot(); // Due to EnsureProjectOnTopJoin, root group can't be Join Group, so DPHyp doesn't change the root group diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index 07963987fc81b32..986947c262575e4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -20,13 +20,14 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.jobs.rewrite.CostBasedRewriteJob; import org.apache.doris.nereids.jobs.rewrite.RewriteJob; -import org.apache.doris.nereids.processor.pre.EliminateLogicalSelectHint; import org.apache.doris.nereids.rules.RuleSet; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.analysis.AdjustAggregateNullableForEmptySet; import org.apache.doris.nereids.rules.analysis.AvgDistinctToSumDivCount; import org.apache.doris.nereids.rules.analysis.CheckAfterRewrite; +import org.apache.doris.nereids.rules.analysis.EliminateGroupByConstant; import org.apache.doris.nereids.rules.analysis.LogicalSubQueryAliasToLogicalProject; +import org.apache.doris.nereids.rules.analysis.NormalizeAggregate; import org.apache.doris.nereids.rules.expression.CheckLegalityAfterRewrite; import org.apache.doris.nereids.rules.expression.ExpressionNormalization; import org.apache.doris.nereids.rules.expression.ExpressionOptimization; @@ -42,6 +43,7 @@ import org.apache.doris.nereids.rules.rewrite.CheckMatchExpression; import org.apache.doris.nereids.rules.rewrite.CheckMultiDistinct; import org.apache.doris.nereids.rules.rewrite.CollectFilterAboveConsumer; +import org.apache.doris.nereids.rules.rewrite.CollectJoinConstraint; import org.apache.doris.nereids.rules.rewrite.CollectProjectAboveConsumer; import org.apache.doris.nereids.rules.rewrite.ColumnPruning; import org.apache.doris.nereids.rules.rewrite.ConvertInnerOrCrossJoin; @@ -52,7 +54,6 @@ import org.apache.doris.nereids.rules.rewrite.EliminateDedupJoinCondition; import org.apache.doris.nereids.rules.rewrite.EliminateEmptyRelation; import org.apache.doris.nereids.rules.rewrite.EliminateFilter; -import org.apache.doris.nereids.rules.rewrite.EliminateGroupByConstant; import org.apache.doris.nereids.rules.rewrite.EliminateLimit; import org.apache.doris.nereids.rules.rewrite.EliminateNotNull; import org.apache.doris.nereids.rules.rewrite.EliminateNullAwareLeftAntiJoin; @@ -69,16 +70,17 @@ import org.apache.doris.nereids.rules.rewrite.InferJoinNotNull; import org.apache.doris.nereids.rules.rewrite.InferPredicates; import org.apache.doris.nereids.rules.rewrite.InferSetOperatorDistinct; +import org.apache.doris.nereids.rules.rewrite.LeadingJoin; import org.apache.doris.nereids.rules.rewrite.MergeFilters; import org.apache.doris.nereids.rules.rewrite.MergeOneRowRelationIntoUnion; import org.apache.doris.nereids.rules.rewrite.MergeProjects; import org.apache.doris.nereids.rules.rewrite.MergeSetOperations; -import org.apache.doris.nereids.rules.rewrite.NormalizeAggregate; import org.apache.doris.nereids.rules.rewrite.NormalizeSort; import org.apache.doris.nereids.rules.rewrite.PruneFileScanPartition; import org.apache.doris.nereids.rules.rewrite.PruneOlapScanPartition; import org.apache.doris.nereids.rules.rewrite.PruneOlapScanTablet; import org.apache.doris.nereids.rules.rewrite.PullUpCteAnchor; +import org.apache.doris.nereids.rules.rewrite.PullUpProjectUnderApply; import org.apache.doris.nereids.rules.rewrite.PushConjunctsIntoEsScan; import org.apache.doris.nereids.rules.rewrite.PushConjunctsIntoJdbcScan; import org.apache.doris.nereids.rules.rewrite.PushFilterInsideJoin; @@ -138,6 +140,10 @@ public class Rewriter extends AbstractBatchJobExecutor { ), // subquery unnesting relay on ExpressionNormalization to extract common factor expression topic("Subquery unnesting", + // after doing NormalizeAggregate in analysis job + // we need run the following 2 rules to make AGG_SCALAR_SUBQUERY_TO_WINDOW_FUNCTION work + bottomUp(new PullUpProjectUnderApply()), + topDown(new PushdownFilterThroughProject()), costBased( custom(RuleType.AGG_SCALAR_SUBQUERY_TO_WINDOW_FUNCTION, AggScalarSubQueryToWindowFunction::new) @@ -158,8 +164,6 @@ public class Rewriter extends AbstractBatchJobExecutor { new ApplyToJoin() ) ), - // we should eliminate hint after "Subquery unnesting" because some hint maybe exist in the CTE or subquery. - custom(RuleType.ELIMINATE_HINT, EliminateLogicalSelectHint::new), topic("Eliminate optimization", bottomUp( new EliminateLimit(), @@ -222,6 +226,15 @@ public class Rewriter extends AbstractBatchJobExecutor { bottomUp(new EliminateNotNull()), topDown(new ConvertInnerOrCrossJoin()) ), + topic("LEADING JOIN", + bottomUp( + new CollectJoinConstraint() + ), + custom(RuleType.LEADING_JOIN, LeadingJoin::new), + bottomUp( + new ExpressionRewrite(CheckLegalityAfterRewrite.INSTANCE) + ) + ), topic("Column pruning and infer predicate", custom(RuleType.COLUMN_PRUNING, ColumnPruning::new), custom(RuleType.INFER_PREDICATES, InferPredicates::new), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/GraphSimplifier.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/GraphSimplifier.java index cf613ac1740ffec..380dd698a45ce43 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/GraphSimplifier.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/GraphSimplifier.java @@ -31,6 +31,7 @@ import org.apache.doris.statistics.Statistics; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.HashMap; @@ -432,7 +433,7 @@ private boolean tryGetSuperset(long bitmap1, long bitmap2, List superset) private Cost calCost(Edge edge, Statistics stats, Statistics leftStats, Statistics rightStats) { LogicalJoin join = edge.getJoin(); - PlanContext planContext = new PlanContext(stats, leftStats, rightStats); + PlanContext planContext = new PlanContext(stats, ImmutableList.of(leftStats, rightStats)); Cost cost; if (JoinUtils.shouldNestedLoopJoin(join)) { PhysicalNestedLoopJoin nestedLoopJoin = new PhysicalNestedLoopJoin<>( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/bitmap/LongBitmap.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/bitmap/LongBitmap.java index 93a81f2580936ab..58529dc41da664f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/bitmap/LongBitmap.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/bitmap/LongBitmap.java @@ -17,7 +17,10 @@ package org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap; +import org.apache.doris.nereids.trees.plans.RelationId; + import java.util.BitSet; +import java.util.Set; /** * This is helper class for some bitmap operation @@ -147,6 +150,14 @@ public static int lowestOneIndex(long bitmap) { return Long.numberOfTrailingZeros(bitmap); } + public static Long computeTableBitmap(Set relationIdSet) { + Long totalBitmap = 0L; + for (RelationId id : relationIdSet) { + totalBitmap = LongBitmap.set(totalBitmap, (id.asInt())); + } + return totalBitmap; + } + public static String toString(long bitmap) { long[] longs = {bitmap}; BitSet bitSet = BitSet.valueOf(longs); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index f0a270bdb61b532..991df341e451fce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -18,6 +18,9 @@ package org.apache.doris.nereids.parser; import org.apache.doris.analysis.ArithmeticExpr.Operator; +import org.apache.doris.analysis.BrokerDesc; +import org.apache.doris.analysis.StorageBackend; +import org.apache.doris.analysis.StorageBackend.StorageType; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.common.Config; import org.apache.doris.common.Pair; @@ -27,20 +30,26 @@ import org.apache.doris.nereids.DorisParser.AliasedQueryContext; import org.apache.doris.nereids.DorisParser.ArithmeticBinaryContext; import org.apache.doris.nereids.DorisParser.ArithmeticUnaryContext; +import org.apache.doris.nereids.DorisParser.ArrayLiteralContext; import org.apache.doris.nereids.DorisParser.ArraySliceContext; import org.apache.doris.nereids.DorisParser.BitOperationContext; import org.apache.doris.nereids.DorisParser.BooleanExpressionContext; import org.apache.doris.nereids.DorisParser.BooleanLiteralContext; import org.apache.doris.nereids.DorisParser.BracketJoinHintContext; import org.apache.doris.nereids.DorisParser.BracketRelationHintContext; +import org.apache.doris.nereids.DorisParser.CollateContext; import org.apache.doris.nereids.DorisParser.ColumnReferenceContext; import org.apache.doris.nereids.DorisParser.CommentJoinHintContext; import org.apache.doris.nereids.DorisParser.CommentRelationHintContext; import org.apache.doris.nereids.DorisParser.ComparisonContext; +import org.apache.doris.nereids.DorisParser.ComplexColTypeContext; +import org.apache.doris.nereids.DorisParser.ComplexColTypeListContext; import org.apache.doris.nereids.DorisParser.ComplexDataTypeContext; import org.apache.doris.nereids.DorisParser.ConstantContext; import org.apache.doris.nereids.DorisParser.CreateRowPolicyContext; import org.apache.doris.nereids.DorisParser.CteContext; +import org.apache.doris.nereids.DorisParser.DateCeilContext; +import org.apache.doris.nereids.DorisParser.DateFloorContext; import org.apache.doris.nereids.DorisParser.Date_addContext; import org.apache.doris.nereids.DorisParser.Date_subContext; import org.apache.doris.nereids.DorisParser.DecimalLiteralContext; @@ -49,6 +58,7 @@ import org.apache.doris.nereids.DorisParser.ElementAtContext; import org.apache.doris.nereids.DorisParser.ExistContext; import org.apache.doris.nereids.DorisParser.ExplainContext; +import org.apache.doris.nereids.DorisParser.ExportContext; import org.apache.doris.nereids.DorisParser.FromClauseContext; import org.apache.doris.nereids.DorisParser.GroupingElementContext; import org.apache.doris.nereids.DorisParser.GroupingSetContext; @@ -69,6 +79,7 @@ import org.apache.doris.nereids.DorisParser.LimitClauseContext; import org.apache.doris.nereids.DorisParser.LogicalBinaryContext; import org.apache.doris.nereids.DorisParser.LogicalNotContext; +import org.apache.doris.nereids.DorisParser.MapLiteralContext; import org.apache.doris.nereids.DorisParser.MultiStatementsContext; import org.apache.doris.nereids.DorisParser.MultipartIdentifierContext; import org.apache.doris.nereids.DorisParser.NamedExpressionContext; @@ -80,9 +91,11 @@ import org.apache.doris.nereids.DorisParser.PredicateContext; import org.apache.doris.nereids.DorisParser.PredicatedContext; import org.apache.doris.nereids.DorisParser.PrimitiveDataTypeContext; -import org.apache.doris.nereids.DorisParser.PropertiesStatmentContext; -import org.apache.doris.nereids.DorisParser.PropertyContext; +import org.apache.doris.nereids.DorisParser.PropertyClauseContext; import org.apache.doris.nereids.DorisParser.PropertyItemContext; +import org.apache.doris.nereids.DorisParser.PropertyItemListContext; +import org.apache.doris.nereids.DorisParser.PropertyKeyContext; +import org.apache.doris.nereids.DorisParser.PropertyValueContext; import org.apache.doris.nereids.DorisParser.QualifiedNameContext; import org.apache.doris.nereids.DorisParser.QueryContext; import org.apache.doris.nereids.DorisParser.QueryOrganizationContext; @@ -98,6 +111,7 @@ import org.apache.doris.nereids.DorisParser.StarContext; import org.apache.doris.nereids.DorisParser.StatementDefaultContext; import org.apache.doris.nereids.DorisParser.StringLiteralContext; +import org.apache.doris.nereids.DorisParser.StructLiteralContext; import org.apache.doris.nereids.DorisParser.SubqueryContext; import org.apache.doris.nereids.DorisParser.SubqueryExpressionContext; import org.apache.doris.nereids.DorisParser.SystemVariableContext; @@ -116,6 +130,7 @@ import org.apache.doris.nereids.DorisParser.WhereClauseContext; import org.apache.doris.nereids.DorisParser.WindowFrameContext; import org.apache.doris.nereids.DorisParser.WindowSpecContext; +import org.apache.doris.nereids.DorisParser.WithRemoteStorageSystemContext; import org.apache.doris.nereids.DorisParserBaseVisitor; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundAlias; @@ -133,6 +148,8 @@ import org.apache.doris.nereids.exceptions.ParseException; import org.apache.doris.nereids.properties.OrderKey; import org.apache.doris.nereids.properties.SelectHint; +import org.apache.doris.nereids.properties.SelectHintLeading; +import org.apache.doris.nereids.properties.SelectHintSetVar; import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.And; import org.apache.doris.nereids.trees.expressions.BitAnd; @@ -177,26 +194,46 @@ import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; import org.apache.doris.nereids.trees.expressions.functions.agg.GroupConcat; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Array; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySlice; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Char; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ConvertTo; +import org.apache.doris.nereids.trees.expressions.functions.scalar.CreateMap; +import org.apache.doris.nereids.trees.expressions.functions.scalar.CreateStruct; +import org.apache.doris.nereids.trees.expressions.functions.scalar.DayCeil; +import org.apache.doris.nereids.trees.expressions.functions.scalar.DayFloor; import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysSub; import org.apache.doris.nereids.trees.expressions.functions.scalar.ElementAt; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncryptKeyRef; +import org.apache.doris.nereids.trees.expressions.functions.scalar.HourCeil; +import org.apache.doris.nereids.trees.expressions.functions.scalar.HourFloor; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteCeil; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteFloor; import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesSub; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthCeil; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthFloor; import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsSub; +import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondCeil; +import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondFloor; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsSub; +import org.apache.doris.nereids.trees.expressions.functions.scalar.WeekCeil; +import org.apache.doris.nereids.trees.expressions.functions.scalar.WeekFloor; import org.apache.doris.nereids.trees.expressions.functions.scalar.WeeksAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.WeeksDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.WeeksSub; +import org.apache.doris.nereids.trees.expressions.functions.scalar.YearCeil; +import org.apache.doris.nereids.trees.expressions.functions.scalar.YearFloor; import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsSub; @@ -214,6 +251,7 @@ import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.literal.SmallIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; import org.apache.doris.nereids.trees.plans.JoinHint; @@ -227,6 +265,7 @@ import org.apache.doris.nereids.trees.plans.commands.DeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; +import org.apache.doris.nereids.trees.plans.commands.ExportCommand; import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.UpdateCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; @@ -251,6 +290,8 @@ 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.types.coercion.CharacterType; import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.policy.FilterType; @@ -353,8 +394,12 @@ public LogicalPlan visitUpdate(UpdateContext ctx) { if (ctx.tableAlias().strictIdentifier() != null) { tableAlias = ctx.tableAlias().getText(); } + Optional cte = Optional.empty(); + if (ctx.cte() != null) { + cte = Optional.ofNullable(withCte(query, ctx.cte())); + } return withExplain(new UpdateCommand(visitMultipartIdentifier(ctx.tableName), tableAlias, - visitUpdateAssignmentSeq(ctx.updateAssignmentSeq()), query), ctx.explain()); + visitUpdateAssignmentSeq(ctx.updateAssignmentSeq()), query, cte), ctx.explain()); } @Override @@ -371,8 +416,73 @@ public LogicalPlan visitDelete(DeleteContext ctx) { if (ctx.tableAlias().strictIdentifier() != null) { tableAlias = ctx.tableAlias().getText(); } - return withExplain(new DeleteCommand(tableName, tableAlias, partitions, query), - ctx.explain()); + Optional cte = Optional.empty(); + if (ctx.cte() != null) { + cte = Optional.ofNullable(withCte(query, ctx.cte())); + } + return withExplain(new DeleteCommand(tableName, tableAlias, partitions, query, cte), ctx.explain()); + } + + @Override + public LogicalPlan visitExport(ExportContext ctx) { + List tableName = visitMultipartIdentifier(ctx.tableName); + List partitions = ctx.partition == null ? ImmutableList.of() : visitIdentifierList(ctx.partition); + String path = parseConstant(ctx.filePath); + String whereSql = null; + if (ctx.whereClause() != null) { + WhereClauseContext whereClauseContext = ctx.whereClause(); + int startIndex = whereClauseContext.start.getStartIndex(); + int stopIndex = whereClauseContext.stop.getStopIndex(); + org.antlr.v4.runtime.misc.Interval interval = new org.antlr.v4.runtime.misc.Interval(startIndex, + stopIndex); + whereSql = whereClauseContext.start.getInputStream().getText(interval); + } + + Map filePropertiesMap = null; + if (ctx.propertyClause() != null) { + filePropertiesMap = visitPropertyClause(ctx.propertyClause()); + } + + BrokerDesc brokerDesc = null; + if (ctx.withRemoteStorageSystem() != null) { + brokerDesc = visitWithRemoteStorageSystem(ctx.withRemoteStorageSystem()); + } + + return new ExportCommand(tableName, partitions, whereSql, path, filePropertiesMap, brokerDesc); + } + + @Override + public Map visitPropertyClause(PropertyClauseContext ctx) { + return visitPropertyItemList(ctx.fileProperties); + } + + @Override + public Map visitPropertyItemList(PropertyItemListContext ctx) { + Builder propertiesMap = ImmutableMap.builder(); + for (PropertyItemContext argument : ctx.properties) { + String key = parsePropertyKey(argument.key); + String value = parsePropertyValue(argument.value); + propertiesMap.put(key, value); + } + return propertiesMap.build(); + } + + @Override + public BrokerDesc visitWithRemoteStorageSystem(WithRemoteStorageSystemContext ctx) { + BrokerDesc brokerDesc = null; + + Map brokerPropertiesMap = visitPropertyItemList(ctx.brokerProperties); + + if (ctx.S3() != null) { + brokerDesc = new BrokerDesc("S3", StorageBackend.StorageType.S3, brokerPropertiesMap); + } else if (ctx.HDFS() != null) { + brokerDesc = new BrokerDesc("HDFS", StorageBackend.StorageType.HDFS, brokerPropertiesMap); + } else if (ctx.LOCAL() != null) { + brokerDesc = new BrokerDesc("HDFS", StorageType.LOCAL, brokerPropertiesMap); + } else if (ctx.BROKER() != null) { + brokerDesc = new BrokerDesc(visitIdentifierOrText(ctx.brokerName), brokerPropertiesMap); + } + return brokerDesc; } /** @@ -394,17 +504,6 @@ public List> visitMultiStatements(MultiState return logicalPlans; } - @Override - public Properties visitPropertiesStatment(PropertiesStatmentContext ctx) { - Builder map = ImmutableMap.builder(); - for (PropertyContext argument : ctx.properties) { - String key = parsePropertyItem(argument.key); - String value = parsePropertyItem(argument.value); - map.put(key, value); - } - return new Properties(map.build()); - } - /* ******************************************************************************************** * Plan parsing * ******************************************************************************************** */ @@ -555,7 +654,6 @@ public LogicalPlan visitRegularQuerySpecification(RegularQuerySpecificationConte ); } selectPlan = withQueryOrganization(selectPlan, ctx.queryOrganization()); - selectPlan = withCte(selectPlan, ctx.cte()); return withSelectHint(selectPlan, selectCtx.selectHint()); }); } @@ -636,14 +734,9 @@ public LogicalPlan visitTableValuedFunction(TableValuedFunctionContext ctx) { return ParserUtils.withOrigin(ctx, () -> { String functionName = ctx.tvfName.getText(); - Builder map = ImmutableMap.builder(); - for (PropertyContext argument : ctx.properties) { - String key = parsePropertyItem(argument.key); - String value = parsePropertyItem(argument.value); - map.put(key, value); - } + Map map = visitPropertyItemList(ctx.properties); LogicalPlan relation = new UnboundTVFRelation(StatementScopeIdGenerator.newRelationId(), - functionName, new Properties(map.build())); + functionName, new Properties(map)); return withTableAlias(relation, ctx.tableAlias()); }); } @@ -836,7 +929,7 @@ public Expression visitArithmeticUnary(ArithmeticUnaryContext ctx) { switch (ctx.operator.getType()) { case DorisParser.PLUS: return e; - case DorisParser.MINUS: + case DorisParser.SUBTRACT: IntegerLiteral zero = new IntegerLiteral(0); return new Subtract(zero, e); case DorisParser.TILDE: @@ -882,7 +975,7 @@ public Expression visitArithmeticBinary(ArithmeticBinaryContext ctx) { Operator op; if (type == DorisParser.PLUS) { op = Operator.ADD; - } else if (type == DorisParser.MINUS) { + } else if (type == DorisParser.SUBTRACT) { op = Operator.SUBTRACT; } else { throw new ParseException("Only supported: " + Operator.ADD + " and " + Operator.SUBTRACT, ctx); @@ -897,11 +990,11 @@ public Expression visitArithmeticBinary(ArithmeticBinaryContext ctx) { return new Multiply(left, right); case DorisParser.SLASH: return new Divide(left, right); - case DorisParser.PERCENT: + case DorisParser.MOD: return new Mod(left, right); case DorisParser.PLUS: return new Add(left, right); - case DorisParser.MINUS: + case DorisParser.SUBTRACT: return new Subtract(left, right); case DorisParser.DIV: return new IntegralDivide(left, right); @@ -1025,6 +1118,64 @@ public Expression visitDate_sub(Date_subContext ctx) { + ", supported time unit: YEAR/MONTH/DAY/HOUR/MINUTE/SECOND", ctx); } + @Override + public Expression visitDateFloor(DateFloorContext ctx) { + Expression timeStamp = (Expression) visit(ctx.timestamp); + Expression amount = (Expression) visit(ctx.unitsAmount); + if (ctx.unit == null) { + // use "SECOND" as unit by default + return new SecondFloor(timeStamp, amount); + } + Expression e = new DateTimeV2Literal(0001L, 01L, 01L, 0L, 0L, 0L, 0L); + + if ("Year".equalsIgnoreCase(ctx.unit.getText())) { + return new YearFloor(timeStamp, amount, e); + } else if ("MONTH".equalsIgnoreCase(ctx.unit.getText())) { + return new MonthFloor(timeStamp, amount, e); + } else if ("WEEK".equalsIgnoreCase(ctx.unit.getText())) { + return new WeekFloor(timeStamp, amount, e); + } else if ("DAY".equalsIgnoreCase(ctx.unit.getText())) { + return new DayFloor(timeStamp, amount, e); + } else if ("Hour".equalsIgnoreCase(ctx.unit.getText())) { + return new HourFloor(timeStamp, amount, e); + } else if ("Minute".equalsIgnoreCase(ctx.unit.getText())) { + return new MinuteFloor(timeStamp, amount, e); + } else if ("Second".equalsIgnoreCase(ctx.unit.getText())) { + return new SecondFloor(timeStamp, amount, e); + } + throw new ParseException("Unsupported time unit: " + ctx.unit + + ", supported time unit: YEAR/MONTH/WEEK/DAY/HOUR/MINUTE/SECOND", ctx); + } + + @Override + public Expression visitDateCeil(DateCeilContext ctx) { + Expression timeStamp = (Expression) visit(ctx.timestamp); + Expression amount = (Expression) visit(ctx.unitsAmount); + if (ctx.unit == null) { + // use "Second" as unit by default + return new SecondCeil(timeStamp, amount); + } + DateTimeV2Literal e = new DateTimeV2Literal(0001L, 01L, 01L, 0L, 0L, 0L, 0L); + + if ("Year".equalsIgnoreCase(ctx.unit.getText())) { + return new YearCeil(timeStamp, amount, e); + } else if ("MONTH".equalsIgnoreCase(ctx.unit.getText())) { + return new MonthCeil(timeStamp, amount, e); + } else if ("WEEK".equalsIgnoreCase(ctx.unit.getText())) { + return new WeekCeil(timeStamp, amount, e); + } else if ("DAY".equalsIgnoreCase(ctx.unit.getText())) { + return new DayCeil(timeStamp, amount, e); + } else if ("Hour".equalsIgnoreCase(ctx.unit.getText())) { + return new HourCeil(timeStamp, amount, e); + } else if ("Minute".equalsIgnoreCase(ctx.unit.getText())) { + return new MinuteCeil(timeStamp, amount, e); + } else if ("Second".equalsIgnoreCase(ctx.unit.getText())) { + return new SecondCeil(timeStamp, amount, e); + } + throw new ParseException("Unsupported time unit: " + ctx.unit + + ", supported time unit: YEAR/MONTH/WEEK/DAY/HOUR/MINUTE/SECOND", ctx); + } + @Override public Expression visitDoublePipes(DorisParser.DoublePipesContext ctx) { return ParserUtils.withOrigin(ctx, () -> { @@ -1085,9 +1236,59 @@ public Expression visitSearchedCase(DorisParser.SearchedCaseContext context) { @Override public Expression visitCast(DorisParser.CastContext ctx) { - DataType dataType = ((DataType) typedVisit(ctx.dataType())).conversion(); - Expression cast = ParserUtils.withOrigin(ctx, () -> - new Cast(getExpression(ctx.expression()), dataType)); + return ParserUtils.withOrigin(ctx, () -> { + DataType dataType = ((DataType) typedVisit(ctx.dataType())).conversion(); + Expression cast = new Cast(getExpression(ctx.expression()), dataType); + return processCast(cast, dataType); + }); + } + + @Override + public UnboundFunction visitExtract(DorisParser.ExtractContext ctx) { + return ParserUtils.withOrigin(ctx, () -> { + String functionName = ctx.field.getText(); + return new UnboundFunction(functionName, false, + Collections.singletonList(getExpression(ctx.source))); + }); + } + + @Override + public Expression visitEncryptKey(DorisParser.EncryptKeyContext ctx) { + return ParserUtils.withOrigin(ctx, () -> { + String db = ctx.dbName == null ? "" : ctx.dbName.getText(); + String key = ctx.keyName.getText(); + return new EncryptKeyRef(new StringLiteral(db), new StringLiteral(key)); + }); + } + + @Override + public Expression visitCharFunction(DorisParser.CharFunctionContext ctx) { + return ParserUtils.withOrigin(ctx, () -> { + String charSet = ctx.charSet == null ? "utf8" : ctx.charSet.getText(); + List arguments = ImmutableList.builder() + .add(new StringLiteral(charSet)) + .addAll(visit(ctx.arguments, Expression.class)) + .build(); + return new Char(arguments); + }); + } + + @Override + public Expression visitConvertCharSet(DorisParser.ConvertCharSetContext ctx) { + return ParserUtils.withOrigin(ctx, + () -> new ConvertTo(getExpression(ctx.argument), new StringLiteral(ctx.charSet.getText()))); + } + + @Override + public Expression visitConvertType(DorisParser.ConvertTypeContext ctx) { + return ParserUtils.withOrigin(ctx, () -> { + DataType dataType = ((DataType) typedVisit(ctx.type)).conversion(); + Expression cast = new Cast(getExpression(ctx.argument), dataType); + return processCast(cast, dataType); + }); + } + + private Expression processCast(Expression cast, DataType dataType) { if (dataType.isStringLikeType() && ((CharacterType) dataType).getLen() >= 0) { List args = ImmutableList.of( cast, @@ -1100,29 +1301,19 @@ public Expression visitCast(DorisParser.CastContext ctx) { } } - @Override - public UnboundFunction visitExtract(DorisParser.ExtractContext ctx) { - return ParserUtils.withOrigin(ctx, () -> { - String functionName = ctx.field.getText(); - return new UnboundFunction(functionName, false, - Collections.singletonList(getExpression(ctx.source))); - }); - } - @Override public Expression visitFunctionCall(DorisParser.FunctionCallContext ctx) { return ParserUtils.withOrigin(ctx, () -> { String functionName = ctx.functionIdentifier().functionNameIdentifier().getText(); boolean isDistinct = ctx.DISTINCT() != null; List params = visit(ctx.expression(), Expression.class); - List orderKeys = visit(ctx.sortItem(), OrderKey.class); if (!orderKeys.isEmpty()) { return parseFunctionWithOrderKeys(functionName, isDistinct, params, orderKeys, ctx); } List unboundStars = ExpressionUtils.collectAll(params, UnboundStar.class::isInstance); - if (unboundStars.size() > 0) { + if (!unboundStars.isEmpty()) { if (functionName.equalsIgnoreCase("count")) { if (unboundStars.size() > 1) { throw new ParseException( @@ -1245,7 +1436,7 @@ public String visitUnitIdentifier(UnitIdentifierContext ctx) { } @Override - public Expression visitTypeConstructor(TypeConstructorContext ctx) { + public Literal visitTypeConstructor(TypeConstructorContext ctx) { String value = ctx.STRING_LITERAL().getText(); value = value.substring(1, value.length() - 1); String type = ctx.type.getText().toUpperCase(); @@ -1301,7 +1492,7 @@ public UnboundSlot visitColumnReference(ColumnReferenceContext ctx) { * Create a NULL literal expression. */ @Override - public Expression visitNullLiteral(NullLiteralContext ctx) { + public Literal visitNullLiteral(NullLiteralContext ctx) { return new NullLiteral(); } @@ -1378,6 +1569,24 @@ private String escapeBackSlash(String str) { return sb.toString(); } + @Override + public Object visitArrayLiteral(ArrayLiteralContext ctx) { + Literal[] items = ctx.items.stream().map(this::typedVisit).toArray(Literal[]::new); + return new Array(items); + } + + @Override + public Object visitMapLiteral(MapLiteralContext ctx) { + Literal[] items = ctx.items.stream().map(this::typedVisit).toArray(Literal[]::new); + return new CreateMap(items); + } + + @Override + public Object visitStructLiteral(StructLiteralContext ctx) { + Literal[] items = ctx.items.stream().map(this::typedVisit).toArray(Literal[]::new); + return new CreateStruct(items); + } + @Override public Expression visitParenthesizedExpression(ParenthesizedExpressionContext ctx) { return getExpression(ctx.expression()); @@ -1511,11 +1720,10 @@ private LogicalPlan withOutFile(LogicalPlan plan, OutFileClauseContext ctx) { if (ctx.format != null) { format = ctx.format.getText(); } - Map properties = Maps.newHashMap(); - for (PropertyContext argument : ctx.properties) { - String key = parseConstant(argument.key.constant()); - String value = parseConstant(argument.value.constant()); - properties.put(key, value); + + Map properties = ImmutableMap.of(); + if (ctx.propertyClause() != null) { + properties = visitPropertyClause(ctx.propertyClause()); } Literal filePath = (Literal) visit(ctx.filePath); return new LogicalFileSink<>(filePath.getStringValue(), format, properties, ImmutableList.of(), plan); @@ -1702,20 +1910,34 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, SelectHintContext hi Map hints = Maps.newLinkedHashMap(); for (HintStatementContext hintStatement : hintContext.hintStatements) { String hintName = hintStatement.hintName.getText().toLowerCase(Locale.ROOT); - Map> parameters = Maps.newLinkedHashMap(); - for (HintAssignmentContext kv : hintStatement.parameters) { - String parameterName = visitIdentifierOrText(kv.key); - Optional value = Optional.empty(); - if (kv.constantValue != null) { - Literal literal = (Literal) visit(kv.constantValue); - value = Optional.ofNullable(literal.toLegacyLiteral().getStringValue()); - } else if (kv.identifierValue != null) { - // maybe we should throw exception when the identifierValue is quoted identifier - value = Optional.ofNullable(kv.identifierValue.getText()); - } - parameters.put(parameterName, value); + switch (hintName) { + case "set_var": + Map> parameters = Maps.newLinkedHashMap(); + for (HintAssignmentContext kv : hintStatement.parameters) { + String parameterName = visitIdentifierOrText(kv.key); + Optional value = Optional.empty(); + if (kv.constantValue != null) { + Literal literal = (Literal) visit(kv.constantValue); + value = Optional.ofNullable(literal.toLegacyLiteral().getStringValue()); + } else if (kv.identifierValue != null) { + // maybe we should throw exception when the identifierValue is quoted identifier + value = Optional.ofNullable(kv.identifierValue.getText()); + } + parameters.put(parameterName, value); + } + hints.put(hintName, new SelectHintSetVar(hintName, parameters)); + break; + case "leading": + List leadingParameters = new ArrayList(); + for (HintAssignmentContext kv : hintStatement.parameters) { + String parameterName = visitIdentifierOrText(kv.key); + leadingParameters.add(parameterName); + } + hints.put(hintName, new SelectHintLeading(hintName, leadingParameters)); + break; + default: + break; } - hints.put(hintName, new SelectHint(hintName, parameters)); } return new LogicalSelectHint<>(hints, logicalPlan); } @@ -1947,7 +2169,14 @@ public Literal visitDecimalLiteral(DecimalLiteralContext ctx) { } } - private String parsePropertyItem(PropertyItemContext item) { + private String parsePropertyKey(PropertyKeyContext item) { + if (item.constant() != null) { + return parseConstant(item.constant()); + } + return item.getText(); + } + + private String parsePropertyValue(PropertyValueContext item) { if (item.constant() != null) { return parseConstant(item.constant()); } @@ -1982,7 +2211,7 @@ private ExplainLevel parseExplainPlanType(PlanTypeContext planTypeContext) { @Override public DataType visitPrimitiveDataType(PrimitiveDataTypeContext ctx) { return ParserUtils.withOrigin(ctx, () -> { - String dataType = ctx.identifier().getText().toLowerCase(Locale.ROOT); + String dataType = ctx.primitiveColType().type.getText().toLowerCase(Locale.ROOT); List l = Lists.newArrayList(dataType); ctx.INTEGER_VALUE().stream().map(ParseTree::getText).forEach(l::add); return DataType.convertPrimitiveFromStrings(l); @@ -1998,13 +2227,30 @@ public DataType visitComplexDataType(ComplexDataTypeContext ctx) { case DorisParser.MAP: return MapType.of(typedVisit(ctx.dataType(0)), typedVisit(ctx.dataType(1))); case DorisParser.STRUCT: - throw new AnalysisException("do not support STRUCT type for Nereids"); + return new StructType(visitComplexColTypeList(ctx.complexColTypeList())); default: throw new AnalysisException("do not support " + ctx.complex.getText() + " type for Nereids"); } }); } + @Override + public List visitComplexColTypeList(ComplexColTypeListContext ctx) { + return ctx.complexColType().stream().map(this::visitComplexColType).collect(ImmutableList.toImmutableList()); + } + + @Override + public StructField visitComplexColType(ComplexColTypeContext ctx) { + String comment; + if (ctx.commentSpec() != null) { + comment = ctx.commentSpec().STRING_LITERAL().getText(); + comment = escapeBackSlash(comment.substring(1, comment.length() - 1)); + } else { + comment = ""; + } + return new StructField(ctx.identifier().getText(), typedVisit(ctx.dataType()), true, comment); + } + private Expression parseFunctionWithOrderKeys(String functionName, boolean isDistinct, List params, List orderKeys, ParserRuleContext ctx) { if (functionName.equalsIgnoreCase("group_concat")) { @@ -2029,4 +2275,9 @@ private String parseConstant(ConstantContext context) { } return context.getText(); } + + @Override + public Object visitCollate(CollateContext ctx) { + return visit(ctx.primaryExpression()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java index 4ce8098a5314e25..0bc228f38696cf9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java @@ -81,7 +81,7 @@ public DataType parseDataType(String dataType) { } public Properties parseProperties(String properties) { - return parse(properties, DorisParser::propertiesStatment); + return parse(properties, DorisParser::propertyItemList); } private T parse(String sql, Function parseFunction) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/EliminateLogicalSelectHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/EliminateLogicalSelectHint.java index af579204d887d21..8ee147c7fe54773 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/EliminateLogicalSelectHint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/EliminateLogicalSelectHint.java @@ -21,12 +21,17 @@ import org.apache.doris.analysis.StringLiteral; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.hint.Hint; +import org.apache.doris.nereids.hint.LeadingHint; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.properties.SelectHint; +import org.apache.doris.nereids.properties.SelectHintLeading; +import org.apache.doris.nereids.properties.SelectHintSetVar; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalSelectHint; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; import org.apache.doris.qe.VariableMgr; @@ -54,7 +59,11 @@ public LogicalPlan visitLogicalSelectHint( for (Entry hint : selectHintPlan.getHints().entrySet()) { String hintName = hint.getKey(); if (hintName.equalsIgnoreCase("SET_VAR")) { - setVar(hint.getValue(), context); + setVar((SelectHintSetVar) hint.getValue(), context); + } else if (hintName.equalsIgnoreCase("ORDERED")) { + ConnectContext.get().getSessionVariable().setDisableJoinReorder(true); + } else if (hintName.equalsIgnoreCase("LEADING")) { + extractLeading((SelectHintLeading) hint.getValue(), context); } else { logger.warn("Can not process select hint '{}' and skip it", hint.getKey()); } @@ -63,7 +72,7 @@ public LogicalPlan visitLogicalSelectHint( return (LogicalPlan) selectHintPlan.child(); } - private void setVar(SelectHint selectHint, StatementContext context) { + private void setVar(SelectHintSetVar selectHint, StatementContext context) { SessionVariable sessionVariable = context.getConnectContext().getSessionVariable(); // set temporary session value, and then revert value in the 'finally block' of StmtExecutor#execute sessionVariable.setIsSingleSetVar(true); @@ -91,4 +100,16 @@ private void setVar(SelectHint selectHint, StatementContext context) { throw new AnalysisException("The nereids is disabled in this sql, fallback to original planner"); } } + + private void extractLeading(SelectHintLeading selectHint, StatementContext context) { + LeadingHint hint = new LeadingHint("Leading", selectHint.getParameters(), selectHint.toString()); + if (context.getHintMap().get("Leading") != null) { + hint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + hint.setErrorMessage("can only have one leading clause"); + } + context.getHintMap().put("Leading", hint); + context.setLeadingJoin(true); + assert (selectHint != null); + assert (context != null); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHint.java index fd91b04196decca..dd8c8af0e9be4bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHint.java @@ -17,10 +17,7 @@ package org.apache.doris.nereids.properties; -import java.util.Map; import java.util.Objects; -import java.util.Optional; -import java.util.stream.Collectors; /** * select hint. @@ -28,34 +25,17 @@ */ public class SelectHint { // e.g. set_var - private final String hintName; - // e.g. query_timeout='1800', exec_mem_limit='2147483648' - private final Map> parameters; + private String hintName; - public SelectHint(String hintName, Map> parameters) { + public SelectHint(String hintName) { this.hintName = Objects.requireNonNull(hintName, "hintName can not be null"); - this.parameters = Objects.requireNonNull(parameters, "parameters can not be null"); } - public String getHintName() { - return hintName; - } - - public Map> getParameters() { - return parameters; + public void setHintName(String hintName) { + this.hintName = hintName; } - @Override - public String toString() { - String kvString = parameters - .entrySet() - .stream() - .map(kv -> - kv.getValue().isPresent() - ? kv.getKey() + "='" + kv.getValue().get() + "'" - : kv.getKey() - ) - .collect(Collectors.joining(", ")); - return hintName + "(" + kvString + ")"; + public String getHintName() { + return hintName; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintLeading.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintLeading.java new file mode 100644 index 000000000000000..1a1921c4c41afa1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintLeading.java @@ -0,0 +1,47 @@ +// 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.properties; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * select hint. + * e.g. set_var(query_timeout='1800', exec_mem_limit='2147483648') + */ +public class SelectHintLeading extends SelectHint { + // e.g. query_timeout='1800', exec_mem_limit='2147483648' + private final List parameters; + + public SelectHintLeading(String hintName, List parameters) { + super(hintName); + this.parameters = parameters; + } + + public List getParameters() { + return parameters; + } + + @Override + public String toString() { + String leadingString = parameters + .stream() + .collect(Collectors.joining(" ")); + return super.getHintName() + "(" + leadingString + ")"; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintSetVar.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintSetVar.java new file mode 100644 index 000000000000000..8c08dcf378fabd6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintSetVar.java @@ -0,0 +1,54 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.properties; + +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * select hint. + * e.g. set_var(query_timeout='1800', exec_mem_limit='2147483648') + */ +public class SelectHintSetVar extends SelectHint { + // e.g. query_timeout='1800', exec_mem_limit='2147483648' + private final Map> parameters; + + public SelectHintSetVar(String hintName, Map> parameters) { + super(hintName); + this.parameters = parameters; + } + + public Map> getParameters() { + return parameters; + } + + @Override + public String toString() { + String kvString = parameters + .entrySet() + .stream() + .map(kv -> + kv.getValue().isPresent() + ? kv.getKey() + "='" + kv.getValue().get() + "'" + : kv.getKey() + ) + .collect(Collectors.joining(", ")); + return super.getHintName() + "(" + kvString + ")"; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index baed7b53ea2a211..4506ac0e39e03b7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -254,8 +254,13 @@ public enum RuleType { CTE_INLINE(RuleTypeClass.REWRITE), REWRITE_CTE_CHILDREN(RuleTypeClass.REWRITE), COLLECT_FILTER_ON_CONSUMER(RuleTypeClass.REWRITE), + + COLLECT_FILTER(RuleTypeClass.REWRITE), + COLLECT_JOIN_CONSTRAINT(RuleTypeClass.REWRITE), COLLECT_PROJECT_ABOVE_CONSUMER(RuleTypeClass.REWRITE), COLLECT_PROJECT_ABOVE_FILTER_CONSUMER(RuleTypeClass.REWRITE), + + LEADING_JOIN(RuleTypeClass.REWRITE), REWRITE_SENTINEL(RuleTypeClass.REWRITE), // topn opts diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index 4981d84f4c6657c..be5c9f460ab584f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -613,7 +613,7 @@ private List flatBoundStar( private List bindSlot( List exprList, List inputs, CascadesContext cascadesContext) { - List slots = new ArrayList<>(); + List slots = new ArrayList<>(exprList.size()); for (E expr : exprList) { E result = bindSlot(expr, inputs, cascadesContext); slots.add(result); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 1fb886c28db9622..f7cb4e7cdb35cf3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -26,6 +26,7 @@ import org.apache.doris.catalog.external.ExternalTable; import org.apache.doris.catalog.external.HMSExternalTable; import org.apache.doris.common.Config; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.Util; import org.apache.doris.nereids.CTEContext; import org.apache.doris.nereids.CascadesContext; @@ -33,6 +34,7 @@ import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.analyzer.UnboundResultSink; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.hint.LeadingHint; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.pattern.MatchingContext; import org.apache.doris.nereids.properties.LogicalProperties; @@ -125,8 +127,15 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe if (cteContext != null) { Optional analyzedCte = cteContext.getAnalyzedCTEPlan(tableName); if (analyzedCte.isPresent()) { - return new LogicalCTEConsumer(unboundRelation.getRelationId(), + LogicalCTEConsumer consumer = new LogicalCTEConsumer(unboundRelation.getRelationId(), cteContext.getCteId(), tableName, analyzedCte.get()); + if (cascadesContext.getStatementContext().isLeadingJoin()) { + LeadingHint leading = (LeadingHint) cascadesContext.getStatementContext() + .getHintMap().get("Leading"); + leading.putRelationIdAndTableName(Pair.of(consumer.getRelationId(), tableName)); + leading.getRelationIdToScanMap().put(consumer.getRelationId(), consumer); + } + return consumer; } } List tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), @@ -147,7 +156,13 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe } // TODO: should generate different Scan sub class according to table's type - return getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); + LogicalPlan scan = getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); + if (cascadesContext.getStatementContext().isLeadingJoin()) { + LeadingHint leading = (LeadingHint) cascadesContext.getStatementContext().getHintMap().get("Leading"); + leading.putRelationIdAndTableName(Pair.of(unboundRelation.getRelationId(), tableName)); + leading.getRelationIdToScanMap().put(unboundRelation.getRelationId(), scan); + } + return scan; } private LogicalPlan bind(CascadesContext cascadesContext, UnboundRelation unboundRelation) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstant.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java similarity index 96% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstant.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java index f5a01fe530543aa..e7fa14e5cb24f6f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstant.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.nereids.rules.rewrite; +package org.apache.doris.nereids.rules.analysis; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; import org.apache.doris.nereids.rules.expression.rules.FoldConstantRule; +import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.Plan; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LogicalSubQueryAliasToLogicalProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LogicalSubQueryAliasToLogicalProject.java index d37e57a02b356bc..9d32a16f492857b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LogicalSubQueryAliasToLogicalProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LogicalSubQueryAliasToLogicalProject.java @@ -17,10 +17,15 @@ package org.apache.doris.nereids.rules.analysis; +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.hint.Hint; +import org.apache.doris.nereids.hint.LeadingHint; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import com.google.common.collect.ImmutableList; @@ -34,8 +39,29 @@ public class LogicalSubQueryAliasToLogicalProject extends OneRewriteRuleFactory @Override public Rule build() { return RuleType.LOGICAL_SUB_QUERY_ALIAS_TO_LOGICAL_PROJECT.build( - logicalSubQueryAlias().then( - alias -> new LogicalProject<>(ImmutableList.copyOf(alias.getOutput()), alias.child())) + logicalSubQueryAlias().thenApply(ctx -> { + LogicalProject project = new LogicalProject<>( + ImmutableList.copyOf(ctx.root.getOutput()), ctx.root.child()); + if (ctx.cascadesContext.getStatementContext().isLeadingJoin()) { + String aliasName = ctx.root.getAlias(); + LeadingHint leading = (LeadingHint) ctx.cascadesContext.getStatementContext() + .getHintMap().get("Leading"); + if (!(project.child() instanceof LogicalRelation)) { + if (leading.getTablelist().contains(aliasName)) { + leading.setStatus(Hint.HintStatus.SYNTAX_ERROR); + leading.setErrorMessage("Leading alias can only be table name alias"); + } + } else { + RelationId id = leading.findRelationIdAndTableName(aliasName); + if (id == null) { + id = ((LogicalRelation) project.child()).getRelationId(); + } + leading.putRelationIdAndTableName(Pair.of(id, aliasName)); + leading.getRelationIdToScanMap().put(id, project); + } + } + return project; + }) ); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java similarity index 85% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregate.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java index 90e997941a98e69..6a141dce7a795cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.nereids.rules.rewrite; +package org.apache.doris.nereids.rules.analysis; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.NormalizeToSlot; +import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -33,6 +35,7 @@ import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableList.Builder; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -185,24 +188,6 @@ public Rule build() { bottomProjects.addAll(aggInputSlots); // build group by exprs List normalizedGroupExprs = groupByToSlotContext.normalizeToUseSlotRef(groupingByExprs); - // build upper project, use two context to do pop up, because agg output maybe contain two part: - // group by keys and agg expressions - List upperProjects = groupByToSlotContext - .normalizeToUseSlotRefWithoutWindowFunction(aggregateOutput); - upperProjects = normalizedAggFuncsToSlotContext.normalizeToUseSlotRefWithoutWindowFunction(upperProjects); - // process Expression like Alias(SlotReference#0)#0 - upperProjects = upperProjects.stream().map(e -> { - if (e instanceof Alias) { - Alias alias = (Alias) e; - if (alias.child() instanceof SlotReference) { - SlotReference slotReference = (SlotReference) alias.child(); - if (slotReference.getExprId().equals(alias.getExprId())) { - return slotReference; - } - } - } - return e; - }).collect(Collectors.toList()); Plan bottomPlan; if (!bottomProjects.isEmpty()) { @@ -211,11 +196,41 @@ public Rule build() { bottomPlan = aggregate.child(); } + List upperProjects = normalizeOutput(aggregateOutput, + groupByToSlotContext, normalizedAggFuncsToSlotContext); + return new LogicalProject<>(upperProjects, aggregate.withNormalized(normalizedGroupExprs, normalizedAggOutput, bottomPlan)); }).toRule(RuleType.NORMALIZE_AGGREGATE); } + private List normalizeOutput(List aggregateOutput, + NormalizeToSlotContext groupByToSlotContext, NormalizeToSlotContext normalizedAggFuncsToSlotContext) { + // build upper project, use two context to do pop up, because agg output maybe contain two part: + // group by keys and agg expressions + List upperProjects = groupByToSlotContext + .normalizeToUseSlotRefWithoutWindowFunction(aggregateOutput); + upperProjects = normalizedAggFuncsToSlotContext.normalizeToUseSlotRefWithoutWindowFunction(upperProjects); + + Builder builder = new ImmutableList.Builder<>(); + for (int i = 0; i < aggregateOutput.size(); i++) { + NamedExpression e = upperProjects.get(i); + // process Expression like Alias(SlotReference#0)#0 + if (e instanceof Alias && e.child(0) instanceof SlotReference) { + SlotReference slotReference = (SlotReference) e.child(0); + if (slotReference.getExprId().equals(e.getExprId())) { + e = slotReference; + } + } + // Make the output ExprId unchanged + if (!e.getExprId().equals(aggregateOutput.get(i).getExprId())) { + e = new Alias(aggregateOutput.get(i).getExprId(), e, aggregateOutput.get(i).getName()); + } + builder.add(e); + } + return builder.build(); + } + private static class CollectNonWindowedAggFuncs extends DefaultExpressionVisitor> { private static final CollectNonWindowedAggFuncs INSTANCE = new CollectNonWindowedAggFuncs(); 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 6dfe95c1167b2be..c28e82f680919ca 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 @@ -21,9 +21,7 @@ import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.rules.Rule; 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.CaseWhen; import org.apache.doris.nereids.trees.expressions.Exists; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.InSubquery; @@ -47,7 +45,6 @@ import java.util.Collection; import java.util.LinkedHashMap; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -68,8 +65,8 @@ public List buildRules() { logicalFilter().thenApply(ctx -> { LogicalFilter filter = ctx.root; - ImmutableList subqueryExprsList = filter.getConjuncts().stream() - .map(e -> (Set) e.collect(SubqueryExpr.class::isInstance)) + ImmutableList> subqueryExprsList = filter.getConjuncts().stream() + .map(e -> (Set) e.collect(SubqueryExpr.class::isInstance)) .collect(ImmutableList.toImmutableList()); if (subqueryExprsList.stream() .flatMap(Collection::stream).noneMatch(SubqueryExpr.class::isInstance)) { @@ -104,8 +101,7 @@ public List buildRules() { tmpPlan = applyPlan; newConjuncts.add(conjunct); } - Set conjuncts = new LinkedHashSet<>(); - conjuncts.addAll(newConjuncts.build()); + Set conjuncts = ImmutableSet.copyOf(newConjuncts.build()); Plan newFilter = new LogicalFilter<>(conjuncts, applyPlan); if (conjuncts.stream().flatMap(c -> c.children().stream()) .anyMatch(MarkJoinSlotReference.class::isInstance)) { @@ -116,36 +112,44 @@ public List buildRules() { return new LogicalFilter<>(conjuncts, applyPlan); }) ), - RuleType.PROJECT_SUBQUERY_TO_APPLY.build( - logicalProject().thenApply(ctx -> { - LogicalProject project = ctx.root; - Set subqueryExprs = new LinkedHashSet<>(); - project.getProjects().stream() - .filter(Alias.class::isInstance) - .map(Alias.class::cast) - .filter(alias -> alias.child() instanceof CaseWhen) - .forEach(alias -> alias.child().children().stream() - .forEach(e -> - subqueryExprs.addAll(e.collect(SubqueryExpr.class::isInstance)))); - if (subqueryExprs.isEmpty()) { - return project; - } - - SubqueryContext context = new SubqueryContext(subqueryExprs); - return new LogicalProject(project.getProjects().stream() - .map(p -> p.withChildren( - new ReplaceSubquery(ctx.statementContext, true) - .replace(p, context))) - .collect(ImmutableList.toImmutableList()), - subqueryToApply( - subqueryExprs.stream().collect(ImmutableList.toImmutableList()), - (LogicalPlan) project.child(), - context.getSubqueryToMarkJoinSlot(), - ctx.cascadesContext, - Optional.empty(), true - )); - }) - ) + 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)) + .collect(ImmutableList.toImmutableList()); + if (subqueryExprsList.stream().flatMap(Collection::stream).count() == 0) { + return project; + } + List oldProjects = ImmutableList.copyOf(project.getProjects()); + ImmutableList.Builder newProjects = new ImmutableList.Builder<>(); + LogicalPlan childPlan = (LogicalPlan) project.child(); + LogicalPlan applyPlan; + for (int i = 0; i < subqueryExprsList.size(); ++i) { + Set subqueryExprs = subqueryExprsList.get(i); + if (subqueryExprs.isEmpty()) { + newProjects.add(oldProjects.get(i)); + continue; + } + + // first step: Replace the subquery in logcialProject's project list + // second step: Replace subquery with LogicalApply + ReplaceSubquery replaceSubquery = + new ReplaceSubquery(ctx.statementContext, true); + SubqueryContext context = new SubqueryContext(subqueryExprs); + Expression newProject = + replaceSubquery.replace(oldProjects.get(i), context); + + applyPlan = subqueryToApply( + subqueryExprs.stream().collect(ImmutableList.toImmutableList()), + childPlan, context.getSubqueryToMarkJoinSlot(), + ctx.cascadesContext, + Optional.of(newProject), true); + childPlan = applyPlan; + newProjects.add((NamedExpression) newProject); + } + + return project.withProjectsAndChild(newProjects.build(), childPlan); + })) ); } @@ -249,28 +253,30 @@ public Expression visitExistsSubquery(Exists exists, SubqueryContext context) { // The result set when NULL is specified in the subquery and still evaluates to TRUE by using EXISTS // When the number of rows returned is empty, agg will return null, so if there is more agg, // it will always consider the returned result to be true + boolean needCreateMarkJoinSlot = isMarkJoin || isProject; MarkJoinSlotReference markJoinSlotReference = null; - if (exists.getQueryPlan().anyMatch(Aggregate.class::isInstance) && isMarkJoin) { + if (exists.getQueryPlan().anyMatch(Aggregate.class::isInstance) && needCreateMarkJoinSlot) { markJoinSlotReference = new MarkJoinSlotReference(statementContext.generateColumnName(), true); - } else if (isMarkJoin) { + } else if (needCreateMarkJoinSlot) { markJoinSlotReference = new MarkJoinSlotReference(statementContext.generateColumnName()); } - if (isMarkJoin) { + if (needCreateMarkJoinSlot) { context.setSubqueryToMarkJoinSlot(exists, Optional.of(markJoinSlotReference)); } - return isMarkJoin ? markJoinSlotReference : BooleanLiteral.TRUE; + return needCreateMarkJoinSlot ? markJoinSlotReference : BooleanLiteral.TRUE; } @Override public Expression visitInSubquery(InSubquery in, SubqueryContext context) { MarkJoinSlotReference markJoinSlotReference = new MarkJoinSlotReference(statementContext.generateColumnName()); - if (isMarkJoin) { + boolean needCreateMarkJoinSlot = isMarkJoin || isProject; + if (needCreateMarkJoinSlot) { context.setSubqueryToMarkJoinSlot(in, Optional.of(markJoinSlotReference)); } - return isMarkJoin ? markJoinSlotReference : BooleanLiteral.TRUE; + return needCreateMarkJoinSlot ? markJoinSlotReference : BooleanLiteral.TRUE; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalization.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalization.java index bd234204b2729f7..6cff3553b43202d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalization.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalization.java @@ -29,7 +29,6 @@ import org.apache.doris.nereids.rules.expression.rules.SimplifyCastRule; import org.apache.doris.nereids.rules.expression.rules.SimplifyNotExprRule; import org.apache.doris.nereids.rules.expression.rules.SupportJavaDateFormatter; -import org.apache.doris.nereids.trees.expressions.Expression; import com.google.common.collect.ImmutableList; @@ -60,10 +59,5 @@ public class ExpressionNormalization extends ExpressionRewrite { public ExpressionNormalization() { super(new ExpressionRuleExecutor(NORMALIZE_REWRITE_RULES)); } - - @Override - public Expression rewrite(Expression expression, ExpressionRewriteContext context) { - return super.rewrite(expression, context); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java index 2a5b83a06f02b67..84abcb9d43c55d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java @@ -25,9 +25,11 @@ 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.types.coercion.CharacterType; -import java.util.Map; +import java.util.List; /** * check cast valid @@ -42,7 +44,7 @@ public Expression visitCast(Cast cast, ExpressionRewriteContext context) { DataType originalType = cast.child().getDataType(); DataType targetType = cast.getDataType(); if (!check(originalType, targetType)) { - throw new AnalysisException("cannot cast " + originalType + " to " + targetType); + throw new AnalysisException("cannot cast " + originalType.toSql() + " to " + targetType.toSql()); } return cast; } @@ -54,17 +56,19 @@ private boolean check(DataType originalType, DataType targetType) { return check(((MapType) originalType).getKeyType(), ((MapType) targetType).getKeyType()) && check(((MapType) originalType).getValueType(), ((MapType) targetType).getValueType()); } else if (originalType instanceof StructType && targetType instanceof StructType) { - Map targetItems = ((StructType) targetType).getItems(); - for (Map.Entry entry : ((StructType) originalType).getItems().entrySet()) { - if (targetItems.containsKey(entry.getKey())) { - if (!check(entry.getValue(), targetItems.get(entry.getKey()))) { - return false; - } - } else { + List targetFields = ((StructType) targetType).getFields(); + List originalFields = ((StructType) originalType).getFields(); + if (targetFields.size() != originalFields.size()) { + return false; + } + for (int i = 0; i < targetFields.size(); i++) { + if (!targetFields.get(i).equals(originalFields.get(i))) { return false; } } return true; + } else if (originalType instanceof CharacterType && targetType instanceof StructType) { + return true; } else { return checkPrimitiveType(originalType, targetType); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/CaseWhenToIf.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/CaseWhenToIf.java new file mode 100644 index 000000000000000..6372338406dd1de --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/CaseWhenToIf.java @@ -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. + +package org.apache.doris.nereids.rules.expression.rules; + +import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.trees.expressions.CaseWhen; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.WhenClause; +import org.apache.doris.nereids.trees.expressions.functions.scalar.If; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; + +/** + * Rewrite rule to convert CASE WHEN to IF. + * For example: + * CASE WHEN a > 1 THEN 1 ELSE 0 END -> IF(a > 1, 1, 0) + */ +public class CaseWhenToIf extends AbstractExpressionRewriteRule { + + public static CaseWhenToIf INSTANCE = new CaseWhenToIf(); + + @Override + public Expression visitCaseWhen(CaseWhen caseWhen, ExpressionRewriteContext context) { + Expression expr = caseWhen; + if (caseWhen.getWhenClauses().size() == 1) { + WhenClause whenClause = caseWhen.getWhenClauses().get(0); + Expression operand = whenClause.getOperand(); + Expression result = whenClause.getResult(); + expr = new If(operand, result, caseWhen.getDefaultValue().orElse(new NullLiteral(result.getDataType()))); + } + // TODO: traverse expr in CASE WHEN / If. + return expr; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java index f381f328fd52f6d..6ed045a3004f294 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java @@ -19,12 +19,12 @@ import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.ExprId; -import org.apache.doris.analysis.LiteralExpr; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.PrimitiveType; -import org.apache.doris.catalog.Type; +import org.apache.doris.catalog.ScalarType; import org.apache.doris.common.IdGenerator; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.nereids.glue.translator.ExpressionTranslator; import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule; @@ -33,9 +33,15 @@ import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.types.CharType; import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.DateTimeV2Type; +import org.apache.doris.nereids.types.DecimalV2Type; +import org.apache.doris.nereids.types.DecimalV3Type; +import org.apache.doris.nereids.types.VarcharType; import org.apache.doris.proto.InternalService; import org.apache.doris.proto.InternalService.PConstantExprResult; +import org.apache.doris.proto.Types.PScalarType; import org.apache.doris.qe.ConnectContext; import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.system.Backend; @@ -46,6 +52,7 @@ import org.apache.doris.thrift.TQueryGlobals; import org.apache.doris.thrift.TQueryOptions; +import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -69,8 +76,8 @@ public class FoldConstantRuleOnBE extends AbstractExpressionRewriteRule { private final IdGenerator idGenerator = ExprId.createGenerator(); @Override - public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) { - Expression expression = FoldConstantRuleOnFE.INSTANCE.rewrite(expr, ctx); + public Expression rewrite(Expression expression, ExpressionRewriteContext ctx) { + expression = FoldConstantRuleOnFE.INSTANCE.rewrite(expression, ctx); return foldByBE(expression, ctx); } @@ -175,26 +182,57 @@ private Map evalOnBE(Map> paramMa if (result.getStatus().getStatusCode() == 0) { for (Entry e : result.getExprResultMapMap().entrySet()) { for (Entry e1 : e.getValue().getMapMap().entrySet()) { + PScalarType pScalarType = e1.getValue().getType(); + TPrimitiveType tPrimitiveType = TPrimitiveType.findByValue(pScalarType.getType()); + PrimitiveType primitiveType = PrimitiveType.fromThrift(Objects.requireNonNull(tPrimitiveType)); Expression ret; if (e1.getValue().getSuccess()) { - TPrimitiveType type = TPrimitiveType.findByValue(e1.getValue().getType().getType()); - Type t = Type.fromPrimitiveType(PrimitiveType.fromThrift(Objects.requireNonNull(type))); - Expr staleExpr = LiteralExpr.create(e1.getValue().getContent(), Objects.requireNonNull(t)); - // Nereids type - DataType t1 = DataType.convertFromString(staleExpr.getType().getPrimitiveType().toString()); - ret = Literal.of(staleExpr.getStringValue()).castTo(t1); + DataType type; + if (PrimitiveType.ARRAY == primitiveType + || PrimitiveType.MAP == primitiveType + || PrimitiveType.STRUCT == primitiveType + || PrimitiveType.AGG_STATE == primitiveType) { + ret = constMap.get(e1.getKey()); + } else { + if (primitiveType == PrimitiveType.CHAR) { + Preconditions.checkState(pScalarType.hasLen(), + "be return char type without len"); + type = CharType.createCharType(pScalarType.getLen()); + } else if (primitiveType == PrimitiveType.VARCHAR) { + Preconditions.checkState(pScalarType.hasLen(), + "be return varchar type without len"); + type = VarcharType.createVarcharType(pScalarType.getLen()); + } else if (primitiveType == PrimitiveType.DECIMALV2) { + type = DecimalV2Type.createDecimalV2Type( + pScalarType.getPrecision(), pScalarType.getScale()); + } else if (primitiveType == PrimitiveType.DATETIMEV2) { + type = DateTimeV2Type.of(pScalarType.getScale()); + } else if (primitiveType == PrimitiveType.DECIMAL32 + || primitiveType == PrimitiveType.DECIMAL64 + || primitiveType == PrimitiveType.DECIMAL128) { + type = DecimalV3Type.createDecimalV3Type( + pScalarType.getPrecision(), pScalarType.getScale()); + } else { + type = DataType.fromCatalogType(ScalarType.createType( + PrimitiveType.fromThrift(tPrimitiveType))); + } + ret = Literal.of(e1.getValue().getContent()).castTo(type); + } } else { ret = constMap.get(e1.getKey()); } + LOG.debug("Be constant folding convert {} to {}", e1.getKey(), ret); resultMap.put(e1.getKey(), ret); } } } else { - LOG.warn("failed to get const expr value from be: {}", result.getStatus().getErrorMsgsList()); + LOG.warn("query {} failed to get const expr value from be: {}", + DebugUtil.printId(context.queryId()), result.getStatus().getErrorMsgsList()); } } catch (Exception e) { - LOG.warn("failed to get const expr value from be: {}", e.getMessage()); + LOG.warn("query {} failed to get const expr value from be: {}", + DebugUtil.printId(context.queryId()), e.getMessage()); } return resultMap; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java index 42f62db904143c1..cef81acd04620df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java @@ -17,7 +17,10 @@ package org.apache.doris.nereids.rules.expression.rules; +import org.apache.doris.catalog.EncryptKey; +import org.apache.doris.catalog.Env; import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule; import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; import org.apache.doris.nereids.trees.expressions.AggregateExpression; @@ -51,6 +54,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentUser; import org.apache.doris.nereids.trees.expressions.functions.scalar.Database; import org.apache.doris.nereids.trees.expressions.functions.scalar.Date; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncryptKeyRef; import org.apache.doris.nereids.trees.expressions.functions.scalar.If; import org.apache.doris.nereids.trees.expressions.functions.scalar.Password; import org.apache.doris.nereids.trees.expressions.functions.scalar.User; @@ -70,9 +74,11 @@ import org.apache.doris.nereids.types.BooleanType; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.GlobalVariable; import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import com.google.common.collect.Lists; import org.apache.commons.codec.digest.DigestUtils; @@ -111,6 +117,29 @@ public Expression visitLiteral(Literal literal, ExpressionRewriteContext context return literal; } + @Override + public Expression visitEncryptKeyRef(EncryptKeyRef encryptKeyRef, ExpressionRewriteContext context) { + String dbName = encryptKeyRef.getDbName(); + ConnectContext connectContext = ConnectContext.get(); + if (Strings.isNullOrEmpty(dbName)) { + dbName = connectContext.getDatabase(); + } + if ("".equals(dbName)) { + throw new AnalysisException("DB " + dbName + "not found"); + } + dbName = ClusterNamespace.getFullName(connectContext.getClusterName(), dbName); + org.apache.doris.catalog.Database database = + Env.getCurrentEnv().getInternalCatalog().getDbNullable(dbName); + if (database == null) { + throw new AnalysisException("DB " + dbName + "not found"); + } + EncryptKey encryptKey = database.getEncryptKey(encryptKeyRef.getEncryptKeyName()); + if (encryptKey == null) { + throw new AnalysisException("Can not found encryptKey" + encryptKeyRef.getEncryptKeyName()); + } + return new StringLiteral(encryptKey.getKeyString()); + } + @Override public Expression visitEqualTo(EqualTo equalTo, ExpressionRewriteContext context) { equalTo = rewriteChildren(equalTo, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java index 69a3499630b0b08..cd9c16dd870d513 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java @@ -46,6 +46,7 @@ import org.apache.doris.nereids.trees.expressions.functions.FunctionBuilder; import org.apache.doris.nereids.trees.expressions.functions.udf.AliasUdfBuilder; import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes; +import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.types.BooleanType; import org.apache.doris.nereids.types.DataType; @@ -116,6 +117,12 @@ public Expression visitUnboundFunction(UnboundFunction unboundFunction, Expressi } } + @Override + public Expression visitBoundFunction(BoundFunction boundFunction, ExpressionRewriteContext context) { + boundFunction = (BoundFunction) super.visitBoundFunction(boundFunction, context); + return TypeCoercionUtils.processBoundFunction(boundFunction); + } + /** * gets the method for calculating the time. * e.g. YEARS_ADD、YEARS_SUB、DAYS_ADD 、DAYS_SUB @@ -271,4 +278,14 @@ public Expression visitMatch(Match match, ExpressionRewriteContext context) { } return match.withChildren(left, right); } + + @Override + public Expression visitCast(Cast cast, ExpressionRewriteContext context) { + cast = (Cast) super.visitCast(cast, context); + // NOTICE: just for compatibility with legacy planner. + if (cast.child().getDataType() instanceof ArrayType || cast.getDataType() instanceof ArrayType) { + TypeCoercionUtils.checkCanCastTo(cast.child().getDataType(), cast.getDataType()); + } + return cast; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicate.java index c66e27e8b20dd68..19574f8f16e057a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicate.java @@ -31,13 +31,21 @@ import org.apache.doris.nereids.trees.expressions.LessThan; import org.apache.doris.nereids.trees.expressions.LessThanEqual; import org.apache.doris.nereids.trees.expressions.NullSafeEqual; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; import org.apache.doris.nereids.trees.expressions.literal.DateLiteral; import org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral; import org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal; import org.apache.doris.nereids.trees.expressions.literal.DateV2Literal; import org.apache.doris.nereids.trees.expressions.literal.DecimalV3Literal; +import org.apache.doris.nereids.trees.expressions.literal.DoubleLiteral; +import org.apache.doris.nereids.trees.expressions.literal.FloatLiteral; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.trees.expressions.literal.SmallIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; import org.apache.doris.nereids.types.BooleanType; import org.apache.doris.nereids.types.DateTimeType; import org.apache.doris.nereids.types.DateTimeV2Type; @@ -46,9 +54,15 @@ import org.apache.doris.nereids.types.DecimalV3Type; import org.apache.doris.nereids.types.coercion.DateLikeType; +import com.google.common.base.Preconditions; + +import java.math.BigDecimal; +import java.math.RoundingMode; + /** * simplify comparison * such as: cast(c1 as DateV2) >= DateV2Literal --> c1 >= DateLiteral + * cast(c1 AS double) > 2.0 --> c1 >= 2 (c1 is integer like type) */ public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule { @@ -65,6 +79,11 @@ public Expression visitComparisonPredicate(ComparisonPredicate cp, ExpressionRew Expression left = rewrite(cp.left(), context); Expression right = rewrite(cp.right(), context); + // float like type: float, double + if (left.getDataType().isFloatLikeType() && right.getDataType().isFloatLikeType()) { + return processFloatLikeTypeCoercion(cp, left, right); + } + // decimalv3 type if (left.getDataType() instanceof DecimalV3Type && right.getDataType() instanceof DecimalV3Type) { @@ -194,6 +213,26 @@ private Expression processDateLikeTypeCoercion(ComparisonPredicate cp, Expressio } } + private Expression processFloatLikeTypeCoercion(ComparisonPredicate comparisonPredicate, + Expression left, Expression right) { + if (left instanceof Literal) { + comparisonPredicate = comparisonPredicate.commute(); + Expression temp = left; + left = right; + right = temp; + } + + if (left instanceof Cast && left.child(0).getDataType().isIntegerLikeType() + && (right instanceof DoubleLiteral || right instanceof FloatLiteral)) { + Cast cast = (Cast) left; + left = cast.child(); + BigDecimal literal = new BigDecimal(((Literal) right).getStringValue()); + return processIntegerDecimalLiteralComparison(comparisonPredicate, left, literal); + } else { + return comparisonPredicate; + } + } + private Expression processDecimalV3TypeCoercion(ComparisonPredicate comparisonPredicate, Expression left, Expression right) { if (left instanceof DecimalV3Literal) { @@ -203,51 +242,113 @@ private Expression processDecimalV3TypeCoercion(ComparisonPredicate comparisonPr right = temp; } - if (left instanceof Cast && left.child(0).getDataType().isDecimalV3Type() - && right instanceof DecimalV3Literal) { + if (left instanceof Cast && right instanceof DecimalV3Literal) { Cast cast = (Cast) left; left = cast.child(); DecimalV3Literal literal = (DecimalV3Literal) right; - if (((DecimalV3Type) left.getDataType()) - .getScale() < ((DecimalV3Type) literal.getDataType()).getScale()) { - int toScale = ((DecimalV3Type) left.getDataType()).getScale(); - if (comparisonPredicate instanceof EqualTo) { - try { - return comparisonPredicate.withChildren(left, new DecimalV3Literal( - (DecimalV3Type) left.getDataType(), literal.getValue().setScale(toScale))); - } catch (ArithmeticException e) { - if (left.nullable()) { - // TODO: the ideal way is to return an If expr like: - // return new If(new IsNull(left), new NullLiteral(BooleanType.INSTANCE), - // BooleanLiteral.of(false)); - // but current fold constant rule can't handle such complex expr with null literal - // before supporting complex conjuncts with null literal folding rules, - // we use a trick way like this: - return new And(new IsNull(left), new NullLiteral(BooleanType.INSTANCE)); - } else { + if (left.getDataType().isDecimalV3Type()) { + if (((DecimalV3Type) left.getDataType()) + .getScale() < ((DecimalV3Type) literal.getDataType()).getScale()) { + int toScale = ((DecimalV3Type) left.getDataType()).getScale(); + if (comparisonPredicate instanceof EqualTo) { + try { + return comparisonPredicate.withChildren(left, + new DecimalV3Literal((DecimalV3Type) left.getDataType(), + literal.getValue().setScale(toScale))); + } catch (ArithmeticException e) { + if (left.nullable()) { + // TODO: the ideal way is to return an If expr like: + // return new If(new IsNull(left), new NullLiteral(BooleanType.INSTANCE), + // BooleanLiteral.of(false)); + // but current fold constant rule can't handle such complex expr with null literal + // before supporting complex conjuncts with null literal folding rules, + // we use a trick way like this: + return new And(new IsNull(left), + new NullLiteral(BooleanType.INSTANCE)); + } else { + return BooleanLiteral.of(false); + } + } + } else if (comparisonPredicate instanceof NullSafeEqual) { + try { + return comparisonPredicate.withChildren(left, + new DecimalV3Literal((DecimalV3Type) left.getDataType(), + literal.getValue().setScale(toScale))); + } catch (ArithmeticException e) { return BooleanLiteral.of(false); } + } else if (comparisonPredicate instanceof GreaterThan + || comparisonPredicate instanceof LessThanEqual) { + return comparisonPredicate.withChildren(left, literal.roundFloor(toScale)); + } else if (comparisonPredicate instanceof LessThan + || comparisonPredicate instanceof GreaterThanEqual) { + return comparisonPredicate.withChildren(left, + literal.roundCeiling(toScale)); } - } else if (comparisonPredicate instanceof NullSafeEqual) { - try { - return comparisonPredicate.withChildren(left, new DecimalV3Literal( - (DecimalV3Type) left.getDataType(), literal.getValue().setScale(toScale))); - } catch (ArithmeticException e) { + } + } else if (left.getDataType().isIntegerLikeType()) { + return processIntegerDecimalLiteralComparison(comparisonPredicate, left, + literal.getValue()); + } + } + + return comparisonPredicate; + } + + private Expression processIntegerDecimalLiteralComparison( + ComparisonPredicate comparisonPredicate, Expression left, BigDecimal literal) { + // we only process isIntegerLikeType, which are tinyint, smallint, int, bigint + if (literal.compareTo(new BigDecimal(Long.MAX_VALUE)) <= 0) { + if (literal.scale() > 0) { + if (comparisonPredicate instanceof EqualTo) { + if (left.nullable()) { + // TODO: the ideal way is to return an If expr like: + // return new If(new IsNull(left), new NullLiteral(BooleanType.INSTANCE), + // BooleanLiteral.of(false)); + // but current fold constant rule can't handle such complex expr with null literal + // before supporting complex conjuncts with null literal folding rules, + // we use a trick way like this: + return new And(new IsNull(left), new NullLiteral(BooleanType.INSTANCE)); + } else { return BooleanLiteral.of(false); } + } else if (comparisonPredicate instanceof NullSafeEqual) { + return BooleanLiteral.of(false); } else if (comparisonPredicate instanceof GreaterThan || comparisonPredicate instanceof LessThanEqual) { - return comparisonPredicate.withChildren(left, literal.roundFloor(toScale)); + return comparisonPredicate.withChildren(left, + convertDecimalToIntegerLikeLiteral( + literal.setScale(0, RoundingMode.FLOOR))); } else if (comparisonPredicate instanceof LessThan || comparisonPredicate instanceof GreaterThanEqual) { - return comparisonPredicate.withChildren(left, literal.roundCeiling(toScale)); + return comparisonPredicate.withChildren(left, + convertDecimalToIntegerLikeLiteral( + literal.setScale(0, RoundingMode.CEILING))); } + } else { + return comparisonPredicate.withChildren(left, + convertDecimalToIntegerLikeLiteral(literal)); } } - return comparisonPredicate; } + private IntegerLikeLiteral convertDecimalToIntegerLikeLiteral(BigDecimal decimal) { + Preconditions.checkArgument( + decimal.scale() == 0 && decimal.compareTo(new BigDecimal(Long.MAX_VALUE)) <= 0, + "decimal literal must have 0 scale and smaller than Long.MAX_VALUE"); + long val = decimal.longValue(); + if (val <= Byte.MAX_VALUE) { + return new TinyIntLiteral((byte) val); + } else if (val <= Short.MAX_VALUE) { + return new SmallIntLiteral((short) val); + } else if (val <= Integer.MAX_VALUE) { + return new IntegerLiteral((int) val); + } else { + return new BigIntLiteral(val); + } + } + private Expression migrateCastToDateTime(Cast cast) { //cast( cast(v as date) as datetime) if v is datetime, set left = v if (cast.child() instanceof Cast diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java index 03962ff75205f7e..f0971c94ba6e841 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java @@ -29,8 +29,8 @@ import org.apache.doris.nereids.properties.RequireProperties; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.analysis.NormalizeAggregate; import org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnFE; -import org.apache.doris.nereids.rules.rewrite.NormalizeAggregate; import org.apache.doris.nereids.trees.expressions.AggregateExpression; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Cast; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEInline.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEInline.java index 46216ec04382904..b0bab1e68115c5d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEInline.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEInline.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.trees.copier.LogicalPlanDeepCopier; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; @@ -37,6 +38,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import java.util.ArrayList; import java.util.List; /** @@ -101,8 +103,10 @@ public Plan visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, LogicalCTEPr for (Slot consumerSlot : cteConsumer.getOutput()) { Slot producerSlot = cteConsumer.getProducerSlot(consumerSlot); ExprId inlineExprId = deepCopierContext.exprIdReplaceMap.get(producerSlot.getExprId()); - Alias alias = new Alias(consumerSlot.getExprId(), producerSlot.withExprId(inlineExprId), - consumerSlot.getName()); + List childrenExprs = new ArrayList<>(); + childrenExprs.add(producerSlot.withExprId(inlineExprId)); + Alias alias = new Alias(consumerSlot.getExprId(), childrenExprs, + consumerSlot.getName(), producerSlot.getQualifier()); projects.add(alias); } return new LogicalProject<>(projects, inlinedPlan); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckDataTypes.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckDataTypes.java index ad889a944c3a7a8..e3ccfa689f104a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckDataTypes.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckDataTypes.java @@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.JsonType; import org.apache.doris.nereids.types.MapType; @@ -39,8 +40,7 @@ */ public class CheckDataTypes implements CustomRewriter { - private static final Set> UNSUPPORTED_TYPE = ImmutableSet.of( - MapType.class, StructType.class, JsonType.class); + private static final Set> UNSUPPORTED_TYPE = ImmutableSet.of(JsonType.class); @Override public Plan rewriteRoot(Plan rootPlan, JobContext jobContext) { @@ -86,7 +86,14 @@ public Expression visit(Expression expr, Void unused) { } private void checkTypes(DataType dataType) { - if (UNSUPPORTED_TYPE.contains(dataType.getClass())) { + if (dataType instanceof ArrayType) { + checkTypes(((ArrayType) dataType).getItemType()); + } else if (dataType instanceof MapType) { + checkTypes(((MapType) dataType).getKeyType()); + checkTypes(((MapType) dataType).getValueType()); + } else if (dataType instanceof StructType) { + ((StructType) dataType).getFields().forEach(f -> this.checkTypes(f.getDataType())); + } else if (UNSUPPORTED_TYPE.contains(dataType.getClass())) { throw new AnalysisException(String.format("type %s is unsupported for Nereids", dataType)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectJoinConstraint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectJoinConstraint.java new file mode 100644 index 000000000000000..a977347a3487f2b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectJoinConstraint.java @@ -0,0 +1,218 @@ +// 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.rewrite; + +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.hint.Hint; +import org.apache.doris.nereids.hint.JoinConstraint; +import org.apache.doris.nereids.hint.LeadingHint; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.JoinType; +import org.apache.doris.nereids.trees.plans.RelationId; +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.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Set; + +/** + * CollectJoinConstraint + */ +public class CollectJoinConstraint implements RewriteRuleFactory { + + @Override + public List buildRules() { + return ImmutableList.of( + logicalRelation().thenApply(ctx -> { + LeadingHint leading = (LeadingHint) ctx.cascadesContext + .getStatementContext().getHintMap().get("Leading"); + if (leading == null) { + return ctx.root; + } else if (leading.isSyntaxError()) { + return ctx.root; + } + return ctx.root; + }).toRule(RuleType.COLLECT_JOIN_CONSTRAINT), + + logicalJoin().thenApply(ctx -> { + LeadingHint leading = (LeadingHint) ctx.cascadesContext + .getStatementContext().getHintMap().get("Leading"); + if (leading == null) { + return ctx.root; + } + LogicalJoin join = ctx.root; + List expressions = join.getHashJoinConjuncts(); + Long totalFilterBitMap = 0L; + Long nonNullableSlotBitMap = 0L; + for (Expression expression : expressions) { + Long nonNullable = calSlotsTableBitMap(leading, expression.getInputSlots(), true); + nonNullableSlotBitMap = LongBitmap.or(nonNullableSlotBitMap, nonNullable); + Long filterBitMap = calSlotsTableBitMap(leading, expression.getInputSlots(), false); + totalFilterBitMap = LongBitmap.or(totalFilterBitMap, filterBitMap); + leading.getFilters().add(Pair.of(filterBitMap, expression)); + } + expressions = join.getOtherJoinConjuncts(); + for (Expression expression : expressions) { + Long nonNullable = calSlotsTableBitMap(leading, expression.getInputSlots(), true); + nonNullableSlotBitMap = LongBitmap.or(nonNullableSlotBitMap, nonNullable); + Long filterBitMap = calSlotsTableBitMap(leading, expression.getInputSlots(), false); + totalFilterBitMap = LongBitmap.or(totalFilterBitMap, filterBitMap); + leading.getFilters().add(Pair.of(filterBitMap, expression)); + } + Long leftHand = LongBitmap.computeTableBitmap(join.left().getInputRelations()); + Long rightHand = LongBitmap.computeTableBitmap(join.right().getInputRelations()); + join.setBitmap(LongBitmap.or(leftHand, rightHand)); + collectJoinConstraintList(leading, leftHand, rightHand, join, totalFilterBitMap, nonNullableSlotBitMap); + + return ctx.root; + }).toRule(RuleType.COLLECT_JOIN_CONSTRAINT), + + logicalFilter().thenApply(ctx -> { + LeadingHint leading = (LeadingHint) ctx.cascadesContext + .getStatementContext().getHintMap().get("Leading"); + if (leading == null) { + return ctx.root; + } + LogicalFilter filter = ctx.root; + Set expressions = filter.getConjuncts(); + for (Expression expression : expressions) { + Long filterBitMap = calSlotsTableBitMap(leading, expression.getInputSlots(), false); + leading.getFilters().add(Pair.of(filterBitMap, expression)); + } + return ctx.root; + }).toRule(RuleType.COLLECT_JOIN_CONSTRAINT), + + logicalProject(logicalOlapScan()).thenApply( + ctx -> { + LeadingHint leading = (LeadingHint) ctx.cascadesContext + .getStatementContext().getHintMap().get("Leading"); + if (leading == null) { + return ctx.root; + } + LogicalProject project = ctx.root; + LogicalOlapScan scan = project.child(); + leading.getRelationIdToScanMap().put(scan.getRelationId(), project); + return ctx.root; + } + ).toRule(RuleType.COLLECT_JOIN_CONSTRAINT) + ); + } + + private void collectJoinConstraintList(LeadingHint leading, Long leftHand, Long rightHand, LogicalJoin join, + Long filterTableBitMap, Long nonNullableSlotBitMap) { + Long totalTables = LongBitmap.or(leftHand, rightHand); + if (join.getJoinType().isInnerJoin()) { + leading.setInnerJoinBitmap(LongBitmap.or(leading.getInnerJoinBitmap(), totalTables)); + return; + } + if (join.getJoinType().isFullOuterJoin()) { + JoinConstraint newJoinConstraint = new JoinConstraint(leftHand, rightHand, leftHand, rightHand, + JoinType.FULL_OUTER_JOIN, false); + leading.getJoinConstraintList().add(newJoinConstraint); + return; + } + boolean isStrict = LongBitmap.isOverlap(nonNullableSlotBitMap, leftHand); + Long minLeftHand = LongBitmap.newBitmapIntersect(filterTableBitMap, leftHand); + Long innerJoinTableBitmap = LongBitmap.and(totalTables, leading.getInnerJoinBitmap()); + Long filterAndInnerBelow = LongBitmap.newBitmapUnion(filterTableBitMap, innerJoinTableBitmap); + Long minRightHand = LongBitmap.newBitmapIntersect(filterAndInnerBelow, rightHand); + for (JoinConstraint other : leading.getJoinConstraintList()) { + if (other.getJoinType() == JoinType.FULL_OUTER_JOIN) { + if (LongBitmap.isOverlap(leftHand, other.getLeftHand()) + || LongBitmap.isOverlap(leftHand, other.getRightHand())) { + minLeftHand = LongBitmap.or(minLeftHand, + other.getLeftHand()); + minLeftHand = LongBitmap.or(minLeftHand, + other.getRightHand()); + } + if (LongBitmap.isOverlap(rightHand, other.getLeftHand()) + || LongBitmap.isOverlap(rightHand, other.getRightHand())) { + minRightHand = LongBitmap.or(minRightHand, + other.getLeftHand()); + minRightHand = LongBitmap.or(minRightHand, + other.getRightHand()); + } + /* Needn't do anything else with the full join */ + continue; + } + + if (LongBitmap.isOverlap(leftHand, other.getRightHand())) { + if (LongBitmap.isOverlap(filterTableBitMap, other.getRightHand()) + && (join.getJoinType().isSemiOrAntiJoin() + || !LongBitmap.isOverlap(nonNullableSlotBitMap, other.getMinRightHand()))) { + minLeftHand = LongBitmap.or(minLeftHand, + other.getLeftHand()); + minLeftHand = LongBitmap.or(minLeftHand, + other.getRightHand()); + } + } + + if (LongBitmap.isOverlap(rightHand, other.getRightHand())) { + if (LongBitmap.isOverlap(filterTableBitMap, other.getRightHand()) + || !LongBitmap.isOverlap(filterTableBitMap, other.getMinLeftHand()) + || join.getJoinType().isSemiOrAntiJoin() + || other.getJoinType().isSemiOrAntiJoin() + || !other.isLhsStrict()) { + minRightHand = LongBitmap.or(minRightHand, other.getLeftHand()); + minRightHand = LongBitmap.or(minRightHand, other.getRightHand()); + } + } + } + + JoinConstraint newJoinConstraint = new JoinConstraint(minLeftHand, minRightHand, leftHand, rightHand, + join.getJoinType(), isStrict); + leading.getJoinConstraintList().add(newJoinConstraint); + } + + private long calSlotsTableBitMap(LeadingHint leading, Set slots, boolean getNotNullable) { + Preconditions.checkArgument(slots.size() != 0); + long bitmap = LongBitmap.newBitmap(); + for (Slot slot : slots) { + if (getNotNullable && slot.nullable()) { + continue; + } + if (!slot.isColumnFromTable()) { + // we can not get info from column not from table + continue; + } + String tableName = leading.getExprIdToTableNameMap().get(slot.getExprId()); + if (tableName == null) { + tableName = slot.getQualifier().get(slot.getQualifier().size() - 1); + leading.getExprIdToTableNameMap().put(slot.getExprId(), tableName); + } + RelationId id = leading.findRelationIdAndTableName(tableName); + if (id == null) { + leading.setStatus(Hint.HintStatus.SYNTAX_ERROR); + leading.setErrorMessage("can not find table: " + tableName); + return bitmap; + } + long currBitmap = LongBitmap.set(bitmap, id.asInt()); + bitmap = LongBitmap.or(bitmap, currBitmap); + } + return bitmap; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java index 85e0f84524caacd..f9ef20a7d093084 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java @@ -217,15 +217,9 @@ private static Optional> fillUpGroupByToOutput( .build()); } - public static final

P pruneOutput(P plan, List originOutput, - Function, P> withPrunedOutput, PruneContext context) { - Optional> prunedOutputs = pruneOutput(originOutput, context); - return prunedOutputs.map(withPrunedOutput).orElse(plan); - } - /** prune output */ - public static Optional> pruneOutput( - List originOutput, PruneContext context) { + public static

P pruneOutput(P plan, List originOutput, + Function, P> withPrunedOutput, PruneContext context) { List prunedOutputs = originOutput.stream() .filter(output -> context.requiredSlots.contains(output.toSlot())) .collect(ImmutableList.toImmutableList()); @@ -235,9 +229,11 @@ public static Optional> pruneOutput( prunedOutputs = ImmutableList.of(minimumColumn); } - return prunedOutputs.equals(originOutput) - ? Optional.empty() - : Optional.of(prunedOutputs); + if (prunedOutputs.equals(originOutput)) { + return plan; + } else { + return withPrunedOutput.apply(prunedOutputs); + } } private

P pruneChildren(P plan) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LeadingJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LeadingJoin.java new file mode 100644 index 000000000000000..0f334384cad822c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LeadingJoin.java @@ -0,0 +1,78 @@ +// 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.rewrite; + +import org.apache.doris.nereids.hint.Hint; +import org.apache.doris.nereids.hint.LeadingHint; +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; +import org.apache.doris.nereids.rules.rewrite.LeadingJoin.LeadingContext; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; + +/** + * Leading join is used to generate leading join and replace original logical join +*/ +public class LeadingJoin extends DefaultPlanRewriter implements CustomRewriter { + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + if (jobContext.getCascadesContext().getStatementContext().isLeadingJoin()) { + Hint leadingHint = jobContext.getCascadesContext().getStatementContext().getHintMap().get("Leading"); + Plan leadingPlan = plan.accept(this, new LeadingContext( + (LeadingHint) leadingHint, ((LeadingHint) leadingHint) + .getLeadingTableBitmap(jobContext.getCascadesContext().getTables()))); + if (leadingHint.isSuccess()) { + jobContext.getCascadesContext().getConnectContext().getSessionVariable().setDisableJoinReorder(true); + } else { + return plan; + } + return leadingPlan; + } + return plan; + } + + @Override + public Plan visit(Plan plan, LeadingContext context) { + Long currentBitMap = LongBitmap.computeTableBitmap(plan.getInputRelations()); + if (LongBitmap.isSubset(currentBitMap, context.totalBitmap) + && plan instanceof LogicalJoin && !context.leading.isSyntaxError()) { + Plan leadingJoin = context.leading.generateLeadingJoinPlan(); + if (context.leading.isSuccess() && leadingJoin != null) { + return leadingJoin; + } + } else { + return (LogicalPlan) super.visit(plan, context); + } + return plan; + } + + /** LeadingContext */ + public static class LeadingContext { + public LeadingHint leading; + public Long totalBitmap; + + public LeadingContext(LeadingHint leading, Long totalBitmap) { + this.leading = leading; + this.totalBitmap = totalBitmap; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownAliasThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownAliasThroughJoin.java index 42649cbac1ead41..7839fcfe95d031e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownAliasThroughJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownAliasThroughJoin.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.MarkJoinSlotReference; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; @@ -45,7 +46,9 @@ public class PushdownAliasThroughJoin extends OneRewriteRuleFactory { public Rule build() { return logicalProject(logicalJoin()) .when(project -> project.getProjects().stream().allMatch(expr -> - (expr instanceof Slot) || (expr instanceof Alias && ((Alias) expr).child() instanceof Slot))) + (expr instanceof Slot && !(expr instanceof MarkJoinSlotReference)) + || (expr instanceof Alias && ((Alias) expr).child() instanceof Slot + && !(((Alias) expr).child() instanceof MarkJoinSlotReference)))) .when(project -> project.getProjects().stream().anyMatch(expr -> expr instanceof Alias)) .then(project -> { LogicalJoin join = project.child(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SemiJoinCommute.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SemiJoinCommute.java index e8f5b9a55d46e8b..54dff35746ba623 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SemiJoinCommute.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SemiJoinCommute.java @@ -29,7 +29,7 @@ public class SemiJoinCommute extends OneRewriteRuleFactory { @Override public Rule build() { return logicalJoin() - .when(join -> join.getJoinType().isRightSemiOrAntiJoin()) + .when(join -> join.getJoinType().isRightSemiOrAntiJoin() || join.getJoinType().isRightOuterJoin()) .whenNot(join -> ConnectContext.get().getSessionVariable().isDisableJoinReorder()) .whenNot(LogicalJoin::hasJoinHint) .whenNot(LogicalJoin::isMarkJoin) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java index e91c685fc2182ef..58b3d105747cb2f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java @@ -148,7 +148,7 @@ protected static Expression removeCastAndAlias(Expression expression) { } protected static boolean containsAllColumn(Expression expression, Set mvColumnNames) { - if (mvColumnNames.contains(Column.getNameWithoutMvPrefix(expression.toSql()))) { + if (mvColumnNames.contains(expression.toSql())) { return true; } if (expression.children().isEmpty()) { @@ -400,7 +400,7 @@ protected SlotContext generateBaseScanExprToMvExpr(LogicalOlapScan mvPlan) { for (Slot mvSlot : mvPlan.getOutputByIndex(mvPlan.getSelectedIndexId())) { boolean isPushed = false; for (Slot baseSlot : mvPlan.getOutput()) { - if (org.apache.doris.analysis.CreateMaterializedViewStmt.isMVColumnAggregate(mvSlot.getName())) { + if (org.apache.doris.analysis.CreateMaterializedViewStmt.isMVColumn(mvSlot.getName())) { continue; } if (baseSlot.toSql().equalsIgnoreCase( @@ -412,7 +412,7 @@ protected SlotContext generateBaseScanExprToMvExpr(LogicalOlapScan mvPlan) { } } if (!isPushed) { - if (org.apache.doris.analysis.CreateMaterializedViewStmt.isMVColumnAggregate(mvSlot.getName())) { + if (org.apache.doris.analysis.CreateMaterializedViewStmt.isMVColumn(mvSlot.getName())) { mvNameToMvSlot.put(normalizeName( org.apache.doris.analysis.CreateMaterializedViewStmt.mvColumnBreaker(mvSlot.getName())), mvSlot); @@ -570,6 +570,9 @@ public Expression visitSlotReference(SlotReference slotReference, Void context) if (baseSlotToMvSlot.containsKey(slotReference)) { return baseSlotToMvSlot.get(slotReference); } + if (mvNameToMvSlot.containsKey(slotReference.toSql())) { + return mvNameToMvSlot.get(slotReference.toSql()); + } return slotReference; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java index 10be3d74caa82d5..6f937cc96d29535 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java @@ -231,9 +231,9 @@ default boolean containsType(Class... types) { * @param that other tree node * @return true if all the tree is equals */ - default boolean deepEquals(TreeNode that) { - Deque thisDeque = new ArrayDeque<>(); - Deque thatDeque = new ArrayDeque<>(); + default boolean deepEquals(TreeNode that) { + Deque> thisDeque = new ArrayDeque<>(); + Deque> thatDeque = new ArrayDeque<>(); thisDeque.push(this); thatDeque.push(that); @@ -244,8 +244,13 @@ default boolean deepEquals(TreeNode that) { return false; } - TreeNode currentNodeThis = thisDeque.pop(); - TreeNode currentNodeThat = thatDeque.pop(); + TreeNode currentNodeThis = thisDeque.pop(); + TreeNode currentNodeThat = thatDeque.pop(); + + // since TreeNode is immutable, use == to short circuit + if (currentNodeThis == currentNodeThat) { + continue; + } // If current nodes are not equal or the number of child nodes differ, return false. if (!currentNodeThis.equals(currentNodeThat) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java index 7fed96245f5cbe4..3ffca9dff47a5e7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java @@ -17,7 +17,10 @@ package org.apache.doris.nereids.trees.copier; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.hint.Hint; +import org.apache.doris.nereids.hint.LeadingHint; import org.apache.doris.nereids.properties.OrderKey; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; @@ -30,6 +33,7 @@ import org.apache.doris.nereids.trees.expressions.SubqueryExpr; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalApply; import org.apache.doris.nereids.trees.plans.logical.LogicalAssertNumRows; @@ -63,6 +67,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -168,6 +173,7 @@ public Plan visitLogicalOlapScan(LogicalOlapScan olapScan, DeepCopierContext con olapScan.getManuallySpecifiedPartitions(), olapScan.getSelectedTabletIds(), olapScan.getHints()); } + updateLeadingRelationIdMap(newOlapScan.getRelationId(), newOlapScan.getTable().getName(), newOlapScan); newOlapScan.getOutput(); context.putRelation(olapScan.getRelationId(), newOlapScan); updateReplaceMapWithOutput(olapScan, newOlapScan, context.exprIdReplaceMap); @@ -194,6 +200,7 @@ public Plan visitLogicalSchemaScan(LogicalSchemaScan schemaScan, DeepCopierConte } LogicalSchemaScan newSchemaScan = new LogicalSchemaScan(StatementScopeIdGenerator.newRelationId(), schemaScan.getTable(), schemaScan.getQualifier()); + updateLeadingRelationIdMap(newSchemaScan.getRelationId(), newSchemaScan.getTable().getName(), newSchemaScan); updateReplaceMapWithOutput(schemaScan, newSchemaScan, context.exprIdReplaceMap); context.putRelation(schemaScan.getRelationId(), newSchemaScan); return newSchemaScan; @@ -206,6 +213,7 @@ public Plan visitLogicalFileScan(LogicalFileScan fileScan, DeepCopierContext con } LogicalFileScan newFileScan = new LogicalFileScan(StatementScopeIdGenerator.newRelationId(), fileScan.getTable(), fileScan.getQualifier()); + updateLeadingRelationIdMap(newFileScan.getRelationId(), fileScan.getTable().getName(), newFileScan); updateReplaceMapWithOutput(fileScan, newFileScan, context.exprIdReplaceMap); context.putRelation(fileScan.getRelationId(), newFileScan); Set conjuncts = fileScan.getConjuncts().stream() @@ -233,6 +241,7 @@ public Plan visitLogicalJdbcScan(LogicalJdbcScan jdbcScan, DeepCopierContext con } LogicalJdbcScan newJdbcScan = new LogicalJdbcScan(StatementScopeIdGenerator.newRelationId(), jdbcScan.getTable(), jdbcScan.getQualifier()); + updateLeadingRelationIdMap(newJdbcScan.getRelationId(), jdbcScan.getTable().getName(), newJdbcScan); updateReplaceMapWithOutput(jdbcScan, newJdbcScan, context.exprIdReplaceMap); context.putRelation(jdbcScan.getRelationId(), newJdbcScan); return newJdbcScan; @@ -246,6 +255,7 @@ public Plan visitLogicalEsScan(LogicalEsScan esScan, DeepCopierContext context) LogicalEsScan newEsScan = new LogicalEsScan(StatementScopeIdGenerator.newRelationId(), esScan.getTable(), esScan.getQualifier()); updateReplaceMapWithOutput(esScan, newEsScan, context.exprIdReplaceMap); + updateLeadingRelationIdMap(newEsScan.getRelationId(), esScan.getTable().getName(), newEsScan); context.putRelation(esScan.getRelationId(), newEsScan); return newEsScan; } @@ -429,6 +439,7 @@ public Plan visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, DeepCopierCo StatementScopeIdGenerator.newRelationId(), cteConsumer.getCteId(), cteConsumer.getName(), consumerToProducerOutputMap, producerToConsumerOutputMap); + updateLeadingRelationIdMap(newCTEConsumer.getRelationId(), cteConsumer.getName(), newCTEConsumer); context.putRelation(cteConsumer.getRelationId(), newCTEConsumer); return newCTEConsumer; } @@ -446,4 +457,13 @@ private void updateReplaceMapWithOutput(Plan oldPlan, Plan newPlan, Map implements // Mask this expression is generated by rule, should be removed. public boolean isGeneratedIsNotNull = false; + private final int depth; + private final int width; + protected Expression(Expression... children) { super(children); + depth = Arrays.stream(children) + .mapToInt(e -> e.depth) + .max().orElse(0) + 1; + width = Arrays.stream(children) + .mapToInt(e -> e.width) + .sum() + (children.length == 0 ? 1 : 0); + checkLimit(); } protected Expression(List children) { super(Optional.empty(), children); + depth = children.stream() + .mapToInt(e -> e.depth) + .max().orElse(0) + 1; + width = children.stream() + .mapToInt(e -> e.width) + .sum() + (children.isEmpty() ? 1 : 0); + checkLimit(); + } + + private void checkLimit() { + if (depth > Config.expr_depth_limit) { + throw new AnalysisException(String.format("Exceeded the maximum depth of an " + + "expression tree (%s).", Config.expr_depth_limit)); + } + if (width > Config.expr_children_limit) { + throw new AnalysisException(String.format("Exceeded the maximum children of an " + + "expression tree (%s).", Config.expr_children_limit)); + } } public Alias alias(String alias) { @@ -98,13 +130,29 @@ private boolean checkInputDataTypesWithExpectType(DataType input, DataType expec && checkInputDataTypesWithExpectType( ((MapType) input).getValueType(), ((MapType) expected).getValueType()); } else if (input instanceof StructType && expected instanceof StructType) { - throw new AnalysisException("not support struct type now."); + List inputFields = ((StructType) input).getFields(); + List expectedFields = ((StructType) expected).getFields(); + if (inputFields.size() != expectedFields.size()) { + return false; + } + for (int i = 0; i < inputFields.size(); i++) { + if (!checkInputDataTypesWithExpectType( + inputFields.get(i).getDataType(), + expectedFields.get(i).getDataType())) { + return false; + } + } + return true; } else { return checkPrimitiveInputDataTypesWithExpectType(input, expected); } } private boolean checkPrimitiveInputDataTypesWithExpectType(DataType input, DataType expected) { + // These type will throw exception when invoke toCatalogDataType() + if (expected instanceof AnyDataType) { + return expected.acceptsType(input); + } // TODO: complete the cast logic like FunctionCallExpr.analyzeImpl boolean legacyCastCompatible = false; try { @@ -148,6 +196,14 @@ public Expression child(int index) { return children.get(index); } + public int getWidth() { + return width; + } + + public int getDepth() { + return depth; + } + @Override public Expression withChildren(List children) { throw new RuntimeException(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java index 0e7ef81cc4e623e..f17489d8a6ac247 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java @@ -27,11 +27,15 @@ import org.apache.doris.nereids.trees.expressions.functions.executable.DateTimeExtractAndTransform; import org.apache.doris.nereids.trees.expressions.functions.executable.ExecutableFunctions; import org.apache.doris.nereids.trees.expressions.functions.executable.NumericArithmetic; +import org.apache.doris.nereids.trees.expressions.functions.executable.TimeRoundSeries; import org.apache.doris.nereids.trees.expressions.literal.DateLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.DecimalV3Type; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.StructType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMultimap; @@ -46,6 +50,7 @@ * An expression evaluator that evaluates the value of an expression. */ public enum ExpressionEvaluator { + INSTANCE; private ImmutableMultimap functions; @@ -106,9 +111,6 @@ private Expression invoke(Expression expression, String fnName, DataType[] args) private FunctionInvoker getFunction(FunctionSignature signature) { Collection functionInvokers = functions.get(signature.getName()); - if (functionInvokers == null) { - return null; - } for (FunctionInvoker candidate : functionInvokers) { DataType[] candidateTypes = candidate.getSignature().getArgTypes(); DataType[] expectedTypes = signature.getArgTypes(); @@ -134,17 +136,17 @@ private void registerFunctions() { if (functions != null) { return; } - ImmutableMultimap.Builder mapBuilder = - new ImmutableMultimap.Builder(); - List classes = ImmutableList.of( + ImmutableMultimap.Builder mapBuilder = new ImmutableMultimap.Builder<>(); + List> classes = ImmutableList.of( DateTimeAcquire.class, DateTimeExtractAndTransform.class, ExecutableFunctions.class, DateLiteral.class, DateTimeArithmetic.class, - NumericArithmetic.class + NumericArithmetic.class, + TimeRoundSeries.class ); - for (Class cls : classes) { + for (Class cls : classes) { for (Method method : cls.getDeclaredMethods()) { ExecFunctionList annotationList = method.getAnnotation(ExecFunctionList.class); if (annotationList != null) { @@ -165,18 +167,39 @@ private void registerFEFunction(ImmutableMultimap.Builder argTypes = new ArrayList<>(); for (String type : annotation.argTypes()) { - if (type.equalsIgnoreCase("DECIMALV3")) { - argTypes.add(DecimalV3Type.WILDCARD); - } else { - argTypes.add(DataType.convertFromString(type)); - } + argTypes.add(replaceDecimalV3WithWildcard(DataType.convertFromString(type))); } FunctionSignature signature = new FunctionSignature(name, - argTypes.toArray(new DataType[argTypes.size()]), returnType); + argTypes.toArray(new DataType[0]), returnType); mapBuilder.put(name, new FunctionInvoker(method, signature)); } } + private DataType replaceDecimalV3WithWildcard(DataType input) { + if (input instanceof ArrayType) { + DataType item = replaceDecimalV3WithWildcard(((ArrayType) input).getItemType()); + if (item == ((ArrayType) input).getItemType()) { + return input; + } + return ArrayType.of(item); + } else if (input instanceof MapType) { + DataType keyType = replaceDecimalV3WithWildcard(((MapType) input).getKeyType()); + DataType valueType = replaceDecimalV3WithWildcard(((MapType) input).getValueType()); + if (keyType == ((MapType) input).getKeyType() && valueType == ((MapType) input).getValueType()) { + return input; + } + return MapType.of(keyType, valueType); + } else if (input instanceof StructType) { + // TODO: support struct type + return input; + } else { + if (input instanceof DecimalV3Type) { + return DecimalV3Type.WILDCARD; + } + return input; + } + } + /** * function invoker. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Properties.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Properties.java index 9c8e22e65b92eac..db0c78c1f78f579 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Properties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Properties.java @@ -52,7 +52,7 @@ public boolean nullable() { @Override public DataType getDataType() throws UnboundException { - return MapType.INSTANCE; + return MapType.SYSTEM_DEFAULT; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WhenClause.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WhenClause.java index dea93d216d18f89..3cc35869907c1dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WhenClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WhenClause.java @@ -111,6 +111,6 @@ public int hashCode() { @Override public String toString() { - return toSql(); + return " WHEN " + left().toString() + " THEN " + right().toString(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignature.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignature.java index 549053f89b8234c..a755e3c517f0fe7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignature.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignature.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.annotation.Developing; -import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.functions.ComputeSignatureHelper.ComputeSignatureChain; import org.apache.doris.nereids.trees.expressions.typecoercion.ImplicitCastInputTypes; import org.apache.doris.nereids.types.ArrayType; @@ -117,7 +116,7 @@ default FunctionSignature computeSignature(FunctionSignature signature) { .get(); } - /** default computeSignature */ + /** use processor to process computeSignature */ static boolean processComplexType(DataType signatureType, DataType realType, BiFunction processor) { if (signatureType instanceof ArrayType && realType instanceof ArrayType) { @@ -127,7 +126,9 @@ static boolean processComplexType(DataType signatureType, DataType realType, return processor.apply(((MapType) signatureType).getKeyType(), ((MapType) realType).getKeyType()) && processor.apply(((MapType) signatureType).getValueType(), ((MapType) realType).getValueType()); } else if (signatureType instanceof StructType && realType instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + // TODO: do not support struct type now + // throw new AnalysisException("do not support struct type now"); + return true; } else { return processor.apply(signatureType, realType); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignatureHelper.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignatureHelper.java index a8e03153ae00fde..223b793a9facbb7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignatureHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignatureHelper.java @@ -73,7 +73,8 @@ private static void collectAnyDataType(DataType sigType, DataType expressionType collectAnyDataType(((MapType) sigType).getKeyType(), NullType.INSTANCE, indexToArgumentTypes); collectAnyDataType(((MapType) sigType).getValueType(), NullType.INSTANCE, indexToArgumentTypes); } else if (sigType instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + // TODO: do not support struct type now + // throw new AnalysisException("do not support struct type now"); } else { if (sigType instanceof AnyDataType && ((AnyDataType) sigType).getIndex() >= 0) { List dataTypes = indexToArgumentTypes.computeIfAbsent( @@ -90,7 +91,8 @@ private static void collectAnyDataType(DataType sigType, DataType expressionType collectAnyDataType(((MapType) sigType).getValueType(), ((MapType) expressionType).getValueType(), indexToArgumentTypes); } else if (sigType instanceof StructType && expressionType instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + // TODO: do not support struct type now + // throw new AnalysisException("do not support struct type now"); } else { if (sigType instanceof AnyDataType && ((AnyDataType) sigType).getIndex() >= 0) { List dataTypes = indexToArgumentTypes.computeIfAbsent( @@ -112,7 +114,8 @@ private static void collectFollowToAnyDataType(DataType sigType, DataType expres collectFollowToAnyDataType(((MapType) sigType).getValueType(), NullType.INSTANCE, indexToArgumentTypes, allNullTypeIndex); } else if (sigType instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + // TODO: do not support struct type now + // throw new AnalysisException("do not support struct type now"); } else { if (sigType instanceof FollowToAnyDataType && allNullTypeIndex.contains(((FollowToAnyDataType) sigType).getIndex())) { @@ -130,7 +133,8 @@ private static void collectFollowToAnyDataType(DataType sigType, DataType expres collectFollowToAnyDataType(((MapType) sigType).getValueType(), ((MapType) expressionType).getValueType(), indexToArgumentTypes, allNullTypeIndex); } else if (sigType instanceof StructType && expressionType instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + // TODO: do not support struct type now + // throw new AnalysisException("do not support struct type now"); } else { if (sigType instanceof FollowToAnyDataType && allNullTypeIndex.contains(((FollowToAnyDataType) sigType).getIndex())) { @@ -149,7 +153,9 @@ private static DataType replaceAnyDataType(DataType dataType, return MapType.of(replaceAnyDataType(((MapType) dataType).getKeyType(), indexToCommonTypes), replaceAnyDataType(((MapType) dataType).getValueType(), indexToCommonTypes)); } else if (dataType instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + // TODO: do not support struct type now + // throw new AnalysisException("do not support struct type now"); + return dataType; } else { if (dataType instanceof AnyDataType && ((AnyDataType) dataType).getIndex() >= 0) { Optional optionalDataType = indexToCommonTypes.get(((AnyDataType) dataType).getIndex()); @@ -177,7 +183,7 @@ public static FunctionSignature implementAnyDataTypeWithIndex( DataType sigType; if (i >= signature.argumentsTypes.size()) { sigType = signature.getVarArgType().orElseThrow( - () -> new IllegalStateException("function arity not match with signature")); + () -> new AnalysisException("function arity not match with signature")); } else { sigType = signature.argumentsTypes.get(i); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/IdenticalSignature.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/IdenticalSignature.java index 829f9125e4236d9..499cd15f7faeaa9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/IdenticalSignature.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/IdenticalSignature.java @@ -18,7 +18,9 @@ package org.apache.doris.nereids.trees.expressions.functions; import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.coercion.AnyDataType; import java.util.List; @@ -33,6 +35,12 @@ public interface IdenticalSignature extends ComputeSignature { static boolean isIdentical(DataType signatureType, DataType realType) { try { // TODO: copy matchesType to DataType + // TODO: resolve AnyDataType invoke toCatalogDataType + if (signatureType instanceof ArrayType) { + if (((ArrayType) signatureType).getItemType() instanceof AnyDataType) { + return false; + } + } return realType.toCatalogDataType().matchesType(signatureType.toCatalogDataType()); } catch (Throwable t) { // the signatureType maybe DataType and can not cast to catalog data type. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ImplicitlyCastableSignature.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ImplicitlyCastableSignature.java index 0a0f2a972f73fec..490d1344b237afe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ImplicitlyCastableSignature.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ImplicitlyCastableSignature.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.catalog.Type; +import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.NullType; import org.apache.doris.nereids.types.coercion.AnyDataType; @@ -51,6 +52,12 @@ static boolean isPrimitiveImplicitlyCastable(DataType signatureType, DataType re } try { // TODO: copy isImplicitlyCastable method to DataType + // TODO: resolve AnyDataType invoke toCatalogDataType + if (signatureType instanceof ArrayType) { + if (((ArrayType) signatureType).getItemType() instanceof AnyDataType) { + return false; + } + } if (Type.isImplicitlyCastable(realType.toCatalogDataType(), signatureType.toCatalogDataType(), true)) { return true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/NullOrIdenticalSignature.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/NullOrIdenticalSignature.java index 5ed0bea656bb094..6533d0fa506e339 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/NullOrIdenticalSignature.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/NullOrIdenticalSignature.java @@ -18,8 +18,10 @@ package org.apache.doris.nereids.trees.expressions.functions; import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.NullType; +import org.apache.doris.nereids.types.coercion.AnyDataType; import java.util.List; @@ -34,6 +36,12 @@ public interface NullOrIdenticalSignature extends ComputeSignature { static boolean isNullOrIdentical(DataType signatureType, DataType realType) { try { // TODO: copy matchesType to DataType + // TODO: resolve AnyDataType invoke toCatalogDataType + if (signatureType instanceof ArrayType) { + if (((ArrayType) signatureType).getItemType() instanceof AnyDataType) { + return false; + } + } return realType instanceof NullType || realType.toCatalogDataType().matchesType(signatureType.toCatalogDataType()); } catch (Throwable t) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java index aafc8d35995ed13..c2c0d43660bb2d8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java @@ -51,7 +51,7 @@ public AnyValue(Expression arg) { * constructor with 1 argument. */ public AnyValue(boolean distinct, Expression arg) { - super("any_value", distinct, arg); + super("any_value", false, arg); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java index 02ac778eea7e4e2..19cd0190bb6e37f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java @@ -44,7 +44,7 @@ public Max(boolean distinct, Expression arg) { } private Max(boolean distinct, boolean alwaysNullable, Expression arg) { - super("max", distinct, alwaysNullable, arg); + super("max", false, alwaysNullable, arg); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java index ef3fd8db3284dde..72b2162eb51f6ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java @@ -45,7 +45,7 @@ public Min(boolean distinct, Expression arg) { } private Min(boolean distinct, boolean alwaysNullable, Expression arg) { - super("min", distinct, alwaysNullable, arg); + super("min", false, alwaysNullable, arg); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/TimeRoundSeries.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/TimeRoundSeries.java index 77fd37502327c86..cde59d856865ff5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/TimeRoundSeries.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/TimeRoundSeries.java @@ -25,10 +25,7 @@ import org.apache.doris.nereids.trees.expressions.literal.DateV2Literal; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; -import java.time.Duration; import java.time.LocalDateTime; -import java.time.temporal.ChronoUnit; -import java.time.temporal.TemporalUnit; /** * executable functions: @@ -48,49 +45,82 @@ enum DATE { SECOND } - private static ChronoUnit dateEnumToUnit(DATE tag) { + // get it's from be/src/vec/functions/function_datetime_floor_ceil.cpp##time_round + private static LocalDateTime getDateCeilOrFloor(DATE tag, LocalDateTime date, int period, LocalDateTime origin, + boolean getCeil) { + DateTimeV2Literal dt = (DateTimeV2Literal) DateTimeV2Literal.fromJavaDateType(date); + DateTimeV2Literal start = (DateTimeV2Literal) DateTimeV2Literal.fromJavaDateType(origin); + long diff = 0; + long trivialPart = 0; + switch (tag) { + case YEAR: { + diff = dt.getYear() - start.getYear(); + trivialPart = (dt.getValue() % 10000000000L) - (start.getValue() % 10000000000L); + break; + } + case MONTH: { + diff = (dt.getYear() - start.getYear()) * 12 + (dt.getMonth() - start.getMonth()); + trivialPart = (dt.getValue() % 100000000L) - (start.getValue() % 100000000L); + break; + } + case DAY: { + diff = dt.getTotalDays() - start.getTotalDays(); + long part2 = dt.getHour() * 3600 + dt.getMinute() * 60 + dt.getSecond(); + long part1 = start.getHour() * 3600 + start.getMinute() * 60 + start.getSecond(); + trivialPart = part2 - part1; + break; + } + case HOUR: { + diff = (dt.getTotalDays() - start.getTotalDays()) * 24 + (dt.getHour() - start.getHour()); + trivialPart = (dt.getMinute() * 60 + dt.getSecond()) + - (start.getMinute() * 60 + start.getSecond()); + break; + } + case MINUTE: { + diff = (dt.getTotalDays() - start.getTotalDays()) * 24 * 60 + (dt.getHour() - start.getHour()) * 60 + + (dt.getMinute() - start.getMinute()); + trivialPart = dt.getSecond() - start.getSecond(); + break; + } + case SECOND: { + diff = (dt.getTotalDays() - start.getTotalDays()) * 24 * 60 * 60 + + (dt.getHour() - start.getHour()) * 60 * 60 + + (dt.getMinute() - start.getMinute()) * 60 + + (dt.getSecond() - start.getSecond()); + trivialPart = 0; + break; + } + default: { + return null; + } + } + if (getCeil) { + diff = diff + (trivialPart > 0 ? 1 : 0); + } else { + diff = diff - (trivialPart < 0 ? 1 : 0); + } + long deltaInsidePeriod = (diff % period + period) % period; + long step = diff - deltaInsidePeriod; + if (getCeil) { + step = step + (deltaInsidePeriod == 0 ? 0 : period); + } switch (tag) { case YEAR: - return ChronoUnit.YEARS; + return ((DateTimeLiteral) start.plusYears(step)).toJavaDateType(); case MONTH: - return ChronoUnit.MONTHS; + return ((DateTimeLiteral) start.plusMonths(step)).toJavaDateType(); case DAY: - return ChronoUnit.DAYS; + return ((DateTimeLiteral) start.plusDays(step)).toJavaDateType(); case HOUR: - return ChronoUnit.HOURS; + return ((DateTimeLiteral) start.plusHours(step)).toJavaDateType(); case MINUTE: - return ChronoUnit.MINUTES; + return ((DateTimeLiteral) start.plusMinutes(step)).toJavaDateType(); + case SECOND: + return ((DateTimeLiteral) start.plusSeconds(step)).toJavaDateType(); default: - return ChronoUnit.SECONDS; + break; } - } - - private static LocalDateTime getDateCeilOrFloor(DATE tag, LocalDateTime date, int period, LocalDateTime origin, - boolean getCeil) { - // Algorithm: - // Firstly, get the unit distance of the two date. - // Secondly, if the origin date is bigger than the date, subtract it to a date before the date by unit. - // Thirdly, re-calculate the distance of the two date. - // Fourthly, get the ceil and floor date of the date by unit and select the corresponding date as the answer. - - // handle origin > date - TemporalUnit unit = dateEnumToUnit(tag); - if (origin.isAfter(date)) { - Duration duration = Duration.between(date, origin); - long hour = Math.abs(duration.get(unit)); - long ceil = ((hour - 1) / period + 1) * period; - origin = origin.minus(ceil, unit); - } - - // get distance - Duration duration = Duration.between(origin, date); - long hour = Math.abs(duration.get(unit)); - long ceil = ((hour - 1) / period + 1) * period; - long floor = hour / period * period; - LocalDateTime floorDate = origin.plus(floor, unit); - LocalDateTime ceilDate = origin.plus(ceil, unit); - - return getCeil ? ceilDate : floorDate; + return null; } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayExcept.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayExcept.java index b87460b766c5767..c1edd9f2e5be071 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayExcept.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayExcept.java @@ -25,7 +25,6 @@ import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.coercion.AnyDataType; -import org.apache.doris.nereids.types.coercion.FollowToAnyDataType; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -40,7 +39,7 @@ public class ArrayExcept extends ScalarFunction implements ExplicitlyCastableSig public static final List SIGNATURES = ImmutableList.of( FunctionSignature.retArgType(0) - .args(ArrayType.of(new AnyDataType(0)), ArrayType.of(new FollowToAnyDataType(0))) + .args(ArrayType.of(new AnyDataType(0)), ArrayType.of(new AnyDataType(0))) ); /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayIntersect.java index 0dc4f38151b7067..0811304e51ff1ff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayIntersect.java @@ -25,7 +25,6 @@ import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.coercion.AnyDataType; -import org.apache.doris.nereids.types.coercion.FollowToAnyDataType; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -40,7 +39,7 @@ public class ArrayIntersect extends ScalarFunction implements ExplicitlyCastable public static final List SIGNATURES = ImmutableList.of( FunctionSignature.retArgType(0) - .args(ArrayType.of(new AnyDataType(0)), ArrayType.of(new FollowToAnyDataType(0))) + .args(ArrayType.of(new AnyDataType(0)), ArrayType.of(new AnyDataType(0))) ); /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayUnion.java index 93e1ca862f107ff..a1152b2c6dcacc8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayUnion.java @@ -25,7 +25,6 @@ import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.coercion.AnyDataType; -import org.apache.doris.nereids.types.coercion.FollowToAnyDataType; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -40,7 +39,7 @@ public class ArrayUnion extends ScalarFunction implements ExplicitlyCastableSign public static final List SIGNATURES = ImmutableList.of( FunctionSignature.retArgType(0) - .args(ArrayType.of(new AnyDataType(0)), ArrayType.of(new FollowToAnyDataType(0))) + .args(ArrayType.of(new AnyDataType(0)), ArrayType.of(new AnyDataType(0))) ); /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Char.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Char.java new file mode 100644 index 000000000000000..77b311835f645f8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Char.java @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.StringType; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'char'. + */ +public class Char extends ScalarFunction + implements ExplicitlyCastableSignature, AlwaysNullable { + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE, IntegerType.INSTANCE)); + + public Char(List varArgs) { + super("char", varArgs); + } + + public Char(Expression... varArgs) { + super("char", varArgs); + } + + /** + * withChildren. + */ + @Override + public Char withChildren(List children) { + return new Char(children); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitChar(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CreateMap.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CreateMap.java new file mode 100644 index 000000000000000..fe966f4e38df01b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CreateMap.java @@ -0,0 +1,92 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.MapType; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'map'. + */ +public class CreateMap extends ScalarFunction + implements ExplicitlyCastableSignature, AlwaysNotNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(MapType.SYSTEM_DEFAULT).args() + ); + + /** + * constructor with 0 or more arguments. + */ + public CreateMap(Expression... varArgs) { + super("map", varArgs); + } + + @Override + public DataType getDataType() { + if (arity() >= 2) { + return MapType.of(child(0).getDataType(), child(1).getDataType()); + } + return MapType.SYSTEM_DEFAULT; + } + + @Override + public void checkLegalityBeforeTypeCoercion() { + if (arity() % 2 != 0) { + throw new AnalysisException("map can't be odd parameters, need even parameters " + this.toSql()); + } + } + + /** + * withChildren. + */ + @Override + public CreateMap withChildren(List children) { + return new CreateMap(children.toArray(new Expression[0])); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitCreateMap(this, context); + } + + @Override + public List getSignatures() { + if (arity() == 0) { + return SIGNATURES; + } else { + return ImmutableList.of(FunctionSignature.of( + getDataType(), + children.stream() + .map(ExpressionTrait::getDataType) + .collect(ImmutableList.toImmutableList()) + )); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CreateNamedStruct.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CreateNamedStruct.java new file mode 100644 index 000000000000000..0184fed65a6e528 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CreateNamedStruct.java @@ -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. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; +import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.StructField; +import org.apache.doris.nereids.types.StructType; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; + +import java.util.List; +import java.util.Set; + +/** + * ScalarFunction 'named_struct'. + */ +public class CreateNamedStruct extends ScalarFunction + implements ExplicitlyCastableSignature, AlwaysNotNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(StructType.SYSTEM_DEFAULT).args() + ); + + /** + * constructor with 0 or more arguments. + */ + public CreateNamedStruct(Expression... varArgs) { + super("named_struct", varArgs); + } + + @Override + public void checkLegalityBeforeTypeCoercion() { + if (arity() % 2 != 0) { + throw new AnalysisException("named_struct can't be odd parameters, need even parameters " + this.toSql()); + } + Set names = Sets.newHashSet(); + for (int i = 0; i < arity(); i = i + 2) { + if (!(child(i) instanceof StringLikeLiteral)) { + throw new AnalysisException("named_struct only allows" + + " constant string parameter in odd position: " + this); + } else { + String name = ((StringLikeLiteral) child(i)).getStringValue(); + if (names.contains(name)) { + throw new AnalysisException("The name of the struct field cannot be repeated." + + " same name fields are " + name); + } else { + names.add(name); + } + } + } + } + + /** + * withChildren. + */ + @Override + public CreateNamedStruct withChildren(List children) { + return new CreateNamedStruct(children.toArray(new Expression[0])); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitCreateNamedStruct(this, context); + } + + @Override + public List getSignatures() { + if (arity() == 0) { + return SIGNATURES; + } else { + ImmutableList.Builder structFields = ImmutableList.builder(); + for (int i = 0; i < arity(); i = i + 2) { + StringLikeLiteral nameLiteral = (StringLikeLiteral) child(i); + structFields.add(new StructField(nameLiteral.getStringValue(), + children.get(i + 1).getDataType(), true, "")); + } + return ImmutableList.of(FunctionSignature.ret(new StructType(structFields.build())) + .args(children.stream().map(ExpressionTrait::getDataType).toArray(DataType[]::new))); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CreateStruct.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CreateStruct.java new file mode 100644 index 000000000000000..cba64fdcce9ecd3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CreateStruct.java @@ -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. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.StructField; +import org.apache.doris.nereids.types.StructType; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'struct'. + */ +public class CreateStruct extends ScalarFunction + implements ExplicitlyCastableSignature, AlwaysNotNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(StructType.SYSTEM_DEFAULT).args() + ); + + /** + * constructor with 0 or more arguments. + */ + public CreateStruct(Expression... varArgs) { + super("struct", varArgs); + } + + /** + * withChildren. + */ + @Override + public CreateStruct withChildren(List children) { + return new CreateStruct(children.toArray(new Expression[0])); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitCreateStruct(this, context); + } + + @Override + public List getSignatures() { + if (arity() == 0) { + return SIGNATURES; + } else { + ImmutableList.Builder structFields = ImmutableList.builder(); + for (int i = 0; i < arity(); i++) { + structFields.add(new StructField(String.valueOf(i + 1), children.get(i).getDataType(), true, "")); + } + return ImmutableList.of(FunctionSignature.ret(new StructType(structFields.build())) + .args(children.stream().map(ExpressionTrait::getDataType).toArray(DataType[]::new))); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncryptKeyRef.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncryptKeyRef.java new file mode 100644 index 000000000000000..7bbe59cfaf07091 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncryptKeyRef.java @@ -0,0 +1,84 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.StringType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.hadoop.util.Lists; + +import java.util.List; + +/** + * ScalarFunction 'EncryptKeyRef'. + */ +public class EncryptKeyRef extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(StringType.INSTANCE).args(StringType.INSTANCE, StringType.INSTANCE) + ); + + /** + * constructor with 2 arguments. + */ + public EncryptKeyRef(Expression arg0, Expression arg1) { + super("encryptKeyRef", Lists.newArrayList(arg0, arg1)); + } + + public EncryptKeyRef(List args) { + super("encryptKeyRef", args); + } + + public String getDbName() { + Preconditions.checkArgument(children.get(0) instanceof StringLikeLiteral); + return ((StringLikeLiteral) children.get(0)).value; + } + + public String getEncryptKeyName() { + Preconditions.checkArgument(children.get(1) instanceof StringLikeLiteral); + return ((StringLikeLiteral) children.get(1)).value; + } + + /** + * withChildren. + */ + @Override + public EncryptKeyRef withChildren(List children) { + Preconditions.checkArgument(children.size() == 2); + return new EncryptKeyRef(children); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitEncryptKeyRef(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapContainsKey.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapContainsKey.java new file mode 100644 index 000000000000000..94e1718f3f77b16 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapContainsKey.java @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BooleanType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.coercion.AnyDataType; +import org.apache.doris.nereids.types.coercion.FollowToAnyDataType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'map_contains_key'. This class is generated by GenerateFunction. + */ +public class MapContainsKey extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BooleanType.INSTANCE) + .args(MapType.of(new AnyDataType(0), AnyDataType.INSTANCE_WITHOUT_INDEX), + new FollowToAnyDataType(0)) + ); + + /** + * constructor with 2 arguments. + */ + public MapContainsKey(Expression arg0, Expression arg1) { + super("map_contains_key", arg0, arg1); + } + + /** + * withChildren. + */ + @Override + public MapContainsKey withChildren(List children) { + Preconditions.checkArgument(children.size() == 2); + return new MapContainsKey(children.get(0), children.get(1)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitMapContainsKey(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapContainsValue.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapContainsValue.java new file mode 100644 index 000000000000000..196a07e136e5c03 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapContainsValue.java @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BooleanType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.coercion.AnyDataType; +import org.apache.doris.nereids.types.coercion.FollowToAnyDataType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'map_contains_value'. This class is generated by GenerateFunction. + */ +public class MapContainsValue extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BooleanType.INSTANCE) + .args(MapType.of(AnyDataType.INSTANCE_WITHOUT_INDEX, new AnyDataType(0)), + new FollowToAnyDataType(0)) + ); + + /** + * constructor with 2 arguments. + */ + public MapContainsValue(Expression arg0, Expression arg1) { + super("map_contains_value", arg0, arg1); + } + + /** + * withChildren. + */ + @Override + public MapContainsValue withChildren(List children) { + Preconditions.checkArgument(children.size() == 2); + return new MapContainsValue(children.get(0), children.get(1)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitMapContainsValue(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapKeys.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapKeys.java new file mode 100644 index 000000000000000..036d518764f6f6c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapKeys.java @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.ArrayType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.coercion.AnyDataType; +import org.apache.doris.nereids.types.coercion.FollowToAnyDataType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'map_keys'. + */ +public class MapKeys extends ScalarFunction + implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(MapType.of( + new AnyDataType(0), AnyDataType.INSTANCE_WITHOUT_INDEX)) + ); + + /** + * constructor with 1 argument. + */ + public MapKeys(Expression arg) { + super("map_keys", arg); + } + + /** + * withChildren. + */ + @Override + public MapKeys withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new MapKeys(children.get(0)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitMapKeys(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapSize.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapSize.java new file mode 100644 index 000000000000000..093cf9189fe2514 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapSize.java @@ -0,0 +1,71 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.coercion.AnyDataType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'map_size'. + */ +public class MapSize extends ScalarFunction + implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BigIntType.INSTANCE).args(MapType.of( + AnyDataType.INSTANCE_WITHOUT_INDEX, AnyDataType.INSTANCE_WITHOUT_INDEX)) + ); + + /** + * constructor with 1 argument. + */ + public MapSize(Expression arg) { + super("map_size", arg); + } + + /** + * withChildren. + */ + @Override + public MapSize withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new MapSize(children.get(0)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitMapSize(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapValues.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapValues.java new file mode 100644 index 000000000000000..0856fbde1902288 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MapValues.java @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.ArrayType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.coercion.AnyDataType; +import org.apache.doris.nereids.types.coercion.FollowToAnyDataType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'map_values'. + */ +public class MapValues extends ScalarFunction + implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(MapType.of( + AnyDataType.INSTANCE_WITHOUT_INDEX, new AnyDataType(0))) + ); + + /** + * constructor with 1 argument. + */ + public MapValues(Expression arg) { + super("map_values", arg); + } + + /** + * withChildren. + */ + @Override + public MapValues withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new MapValues(children.get(0)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitMapValues(this, context); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StructElement.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StructElement.java new file mode 100644 index 000000000000000..d21b365947c65db --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/StructElement.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.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.SearchSignature; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral; +import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.StructType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'struct_element'. + */ +public class StructElement extends ScalarFunction + implements ExplicitlyCastableSignature, AlwaysNullable { + + /** + * constructor with 0 or more arguments. + */ + public StructElement(Expression arg0, Expression arg1) { + super("struct_element", arg0, arg1); + } + + @Override + public void checkLegalityBeforeTypeCoercion() { + if (!(child(0).getDataType() instanceof StructType)) { + SearchSignature.throwCanNotFoundFunctionException(this.getName(), this.getArguments()); + } + if (!(child(1) instanceof StringLikeLiteral || child(1) instanceof IntegerLikeLiteral)) { + throw new AnalysisException("struct_element only allows" + + " constant int or string second parameter: " + this.toSql()); + } + } + + /** + * withChildren. + */ + @Override + public StructElement withChildren(List children) { + Preconditions.checkArgument(children.size() == 2, "children size should be 2"); + return new StructElement(children.get(0), children.get(1)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitStructElement(this, context); + } + + @Override + public List getSignatures() { + StructType structArgType = (StructType) child(0).getDataType(); + DataType retType; + if (child(1) instanceof IntegerLikeLiteral) { + int offset = ((IntegerLikeLiteral) child(1)).getIntValue(); + if (offset <= 0 || offset > structArgType.getFields().size()) { + throw new AnalysisException("the specified field index out of bound: " + this.toSql()); + } else { + retType = structArgType.getFields().get(offset - 1).getDataType(); + } + } else if (child(1) instanceof StringLikeLiteral) { + String name = ((StringLikeLiteral) child(1)).getStringValue(); + if (!structArgType.getNameToFields().containsKey(name)) { + throw new AnalysisException("the specified field name " + name + " was not found: " + this.toSql()); + } else { + retType = structArgType.getNameToFields().get(name).getDataType(); + } + } else { + throw new AnalysisException("struct_element only allows" + + " constant int or string second parameter: " + this.toSql()); + } + return ImmutableList.of(FunctionSignature.ret(retType).args(structArgType, child(1).getDataType())); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/ArrayLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/ArrayLiteral.java index a669385b793e3c8..5fc4ded2cb09fd4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/ArrayLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/ArrayLiteral.java @@ -19,8 +19,6 @@ import org.apache.doris.analysis.LiteralExpr; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Array; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; @@ -32,6 +30,7 @@ /** ArrayLiteral */ public class ArrayLiteral extends Literal { + private final List items; public ArrayLiteral(List items) { @@ -63,7 +62,7 @@ public LiteralExpr toLegacyLiteral() { @Override public String toString() { String items = this.items.stream() - .map(item -> item.toString()) + .map(Literal::toString) .collect(Collectors.joining(", ")); return "array(" + items + ")"; } @@ -85,6 +84,6 @@ private static DataType computeDataType(List items) { if (items.isEmpty()) { return ArrayType.SYSTEM_DEFAULT; } - return new Array(items.toArray(new Expression[0])).getDataType(); + return ArrayType.of(items.get(0).dataType); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java index f75e2d81dad2f35..dfb7177ebcaff9d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java @@ -206,15 +206,15 @@ public long getDay() { return day; } - public Expression plusDays(int days) { + public Expression plusDays(long days) { return fromJavaDateType(DateUtils.getTime(DATE_FORMATTER, getStringValue()).plusDays(days)); } - public Expression plusMonths(int months) { + public Expression plusMonths(long months) { return fromJavaDateType(DateUtils.getTime(DATE_FORMATTER, getStringValue()).plusMonths(months)); } - public Expression plusYears(int years) { + public Expression plusYears(long years) { return fromJavaDateType(DateUtils.getTime(DATE_FORMATTER, getStringValue()).plusYears(years)); } @@ -222,6 +222,34 @@ public LocalDateTime toJavaDateType() { return LocalDateTime.of(((int) getYear()), ((int) getMonth()), ((int) getDay()), 0, 0, 0); } + public long getTotalDays() { + return calculateDays(this.year, this.month, this.day); + } + + // calculate the number of days from year 0000-00-00 to year-month-day + private long calculateDays(long year, long month, long day) { + long totalDays = 0; + long y = year; + + if (year == 0 && month == 0) { + return 0; + } + + /* Cast to int to be able to handle month == 0 */ + totalDays = 365 * y + 31 * (month - 1) + day; + if (month <= 2) { + // No leap year + y--; + } else { + // This is great!!! + // 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 + // 0, 0, 3, 3, 4, 4, 5, 5, 5, 6, 7, 8 + totalDays -= (month * 4 + 23) / 10; + } + // Every 400 year has 97 leap year, 100, 200, 300 are not leap year. + return totalDays + y / 4 - y / 100 + y / 400; + } + public static Expression fromJavaDateType(LocalDateTime dateTime) { return isDateOutOfRange(dateTime) ? new NullLiteral(DateType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java index 2db3dacb49b5b7d..bf41c967cc60f9c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java @@ -316,23 +316,23 @@ public LiteralExpr toLegacyLiteral() { return new org.apache.doris.analysis.DateLiteral(year, month, day, hour, minute, second, Type.DATETIME); } - public Expression plusYears(int years) { + public Expression plusYears(long years) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER, getStringValue()).plusYears(years)); } - public Expression plusMonths(int months) { + public Expression plusMonths(long months) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER, getStringValue()).plusMonths(months)); } - public Expression plusDays(int days) { + public Expression plusDays(long days) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER, getStringValue()).plusDays(days)); } - public Expression plusHours(int hours) { + public Expression plusHours(long hours) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER, getStringValue()).plusHours(hours)); } - public Expression plusMinutes(int minutes) { + public Expression plusMinutes(long minutes) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER, getStringValue()).plusMinutes(minutes)); } 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 ef09399a5aa8319..c65c921ac7d666b 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 @@ -82,31 +82,31 @@ public String getStringValue() { } @Override - public Expression plusYears(int years) { + public Expression plusYears(long years) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) .plusYears(years), getDataType().getScale()); } @Override - public Expression plusMonths(int months) { + public Expression plusMonths(long months) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) .plusMonths(months), getDataType().getScale()); } @Override - public Expression plusDays(int days) { + public Expression plusDays(long days) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) .plusDays(days), getDataType().getScale()); } @Override - public Expression plusHours(int hours) { + public Expression plusHours(long hours) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) .plusHours(hours), getDataType().getScale()); } @Override - public Expression plusMinutes(int minutes) { + public Expression plusMinutes(long minutes) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) .plusMinutes(minutes), getDataType().getScale()); } @@ -117,7 +117,7 @@ public Expression plusSeconds(long seconds) { .plusSeconds(seconds), getDataType().getScale()); } - public Expression plusMicroSeconds(int microSeconds) { + public Expression plusMicroSeconds(long microSeconds) { return fromJavaDateType(DateUtils.getTime(DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) .plusNanos(microSeconds * 1000L), getDataType().getScale()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java index 928b7ca0f24cf0b..32164c977c42c68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java @@ -50,15 +50,15 @@ public R accept(ExpressionVisitor visitor, C context) { return visitor.visitDateV2Literal(this, context); } - public Expression plusDays(int days) { + public Expression plusDays(long days) { return fromJavaDateType(DateUtils.getTime(DATE_FORMATTER, getStringValue()).plusDays(days)); } - public Expression plusMonths(int months) { + public Expression plusMonths(long months) { return fromJavaDateType(DateUtils.getTime(DATE_FORMATTER, getStringValue()).plusMonths(months)); } - public Expression plusYears(int years) { + public Expression plusYears(long years) { return fromJavaDateType(DateUtils.getTime(DATE_FORMATTER, getStringValue()).plusYears(years)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DoubleLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DoubleLiteral.java index 34b8ca36fbc8a58..b155fe307563c89 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DoubleLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DoubleLiteral.java @@ -23,6 +23,8 @@ import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DoubleType; +import java.text.NumberFormat; + /** * Double literal */ @@ -49,4 +51,16 @@ public R accept(ExpressionVisitor visitor, C context) { public LiteralExpr toLegacyLiteral() { return new FloatLiteral(value, Type.DOUBLE); } + + @Override + public String toString() { + NumberFormat nf = NumberFormat.getInstance(); + nf.setGroupingUsed(false); + return nf.format(value); + } + + @Override + public String getStringValue() { + return toString(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/FloatLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/FloatLiteral.java index 4fff7445efae4d6..95549901dda2c26 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/FloatLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/FloatLiteral.java @@ -22,6 +22,8 @@ import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.FloatType; +import java.text.NumberFormat; + /** * float type literal */ @@ -48,4 +50,11 @@ public R accept(ExpressionVisitor visitor, C context) { public LiteralExpr toLegacyLiteral() { return new org.apache.doris.analysis.FloatLiteral((double) value, Type.FLOAT); } + + @Override + public String getStringValue() { + NumberFormat nf = NumberFormat.getInstance(); + nf.setGroupingUsed(false); + return nf.format(value); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/MapLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/MapLiteral.java new file mode 100644 index 000000000000000..12b0cb8ca733cd5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/MapLiteral.java @@ -0,0 +1,123 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.literal; + +import org.apache.doris.analysis.LiteralExpr; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.NullType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** MapLiteral */ +public class MapLiteral extends Literal { + + private final List keys; + private final List values; + + public MapLiteral() { + super(MapType.SYSTEM_DEFAULT); + this.keys = ImmutableList.of(); + this.values = ImmutableList.of(); + } + + public MapLiteral(List keys, List values) { + super(computeDataType(keys, values)); + this.keys = ImmutableList.copyOf(Objects.requireNonNull(keys, "keys should not be null")); + this.values = ImmutableList.copyOf(Objects.requireNonNull(values, "values should not be null")); + Preconditions.checkArgument(keys.size() == values.size(), + "key size %s is not equal to value size %s", keys.size(), values.size()); + } + + @Override + public List> getValue() { + return ImmutableList.of(keys, values); + } + + @Override + public LiteralExpr toLegacyLiteral() { + if (keys.isEmpty()) { + return new org.apache.doris.analysis.MapLiteral(); + } else { + List keyExprs = keys.stream() + .map(Literal::toLegacyLiteral) + .collect(Collectors.toList()); + List valueExprs = values.stream() + .map(Literal::toLegacyLiteral) + .collect(Collectors.toList()); + try { + return new org.apache.doris.analysis.MapLiteral( + getDataType().toCatalogDataType(), keyExprs, valueExprs); + } catch (Throwable t) { + throw new AnalysisException(t.getMessage(), t); + } + } + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("map("); + if (!keys.isEmpty()) { + sb.append(keys.get(0).toString()).append(", ").append(values.get(0).toString()); + } + for (int i = 1; i < keys.size(); i++) { + sb.append(", ").append(keys.get(i).toString()).append(",").append(values.get(i).toString()); + } + sb.append(")"); + return sb.toString(); + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("map("); + if (!keys.isEmpty()) { + sb.append(keys.get(0).toSql()).append(", ").append(values.get(0).toSql()); + } + for (int i = 1; i < keys.size(); i++) { + sb.append(", ").append(keys.get(i).toSql()).append(",").append(values.get(i).toSql()); + } + sb.append(")"); + return sb.toString(); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitMapLiteral(this, context); + } + + private static DataType computeDataType(List keys, List values) { + DataType keyType = NullType.INSTANCE; + DataType valueType = NullType.INSTANCE; + if (!keys.isEmpty()) { + keyType = keys.get(0).dataType; + } + if (!values.isEmpty()) { + valueType = values.get(0).dataType; + } + return MapType.of(keyType, valueType); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StringLikeLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StringLikeLiteral.java index 480ac996a165924..b9cdd7eced7a760 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StringLikeLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StringLikeLiteral.java @@ -43,4 +43,9 @@ public double getDouble() { } return (double) v; } + + @Override + public String toString() { + return "'" + value + "'"; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StringLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StringLiteral.java index 588adbc0aa4d2ac..603749c60b376d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StringLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StringLiteral.java @@ -52,9 +52,4 @@ public R accept(ExpressionVisitor visitor, C context) { public LiteralExpr toLegacyLiteral() { return new org.apache.doris.analysis.StringLiteral(value); } - - @Override - public String toString() { - return "'" + value + "'"; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/VarcharLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/VarcharLiteral.java index 40f3f8f8addd483..75161dd00042a89 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/VarcharLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/VarcharLiteral.java @@ -50,9 +50,4 @@ public R accept(ExpressionVisitor visitor, C context) { public LiteralExpr toLegacyLiteral() { return new StringLiteral(value); } - - @Override - public String toString() { - return "'" + value + "'"; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/DefaultExpressionRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/DefaultExpressionRewriter.java index 6c25fcbe90bff72..f6e3fc6464747b6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/DefaultExpressionRewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/DefaultExpressionRewriter.java @@ -35,7 +35,7 @@ public Expression visit(Expression expr, C context) { /** rewrite */ public static final Expression rewrite(ExpressionVisitor rewriter, Expression expr, C context) { - List newChildren = new ArrayList<>(); + List newChildren = new ArrayList<>(expr.arity()); boolean hasNewChildren = false; for (Expression child : expr.children()) { Expression newChild = child.accept(rewriter, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java index 0682e255d887ed5..a9aa0047baf6103 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java @@ -101,6 +101,7 @@ import org.apache.doris.nereids.trees.expressions.literal.Interval; import org.apache.doris.nereids.trees.expressions.literal.LargeIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.MapLiteral; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.literal.SmallIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; @@ -294,6 +295,10 @@ public R visitArrayLiteral(ArrayLiteral arrayLiteral, C context) { return visitLiteral(arrayLiteral, context); } + public R visitMapLiteral(MapLiteral mapLiteral, C context) { + return visitLiteral(mapLiteral, context); + } + public R visitCompoundPredicate(CompoundPredicate compoundPredicate, C context) { return visitBinaryOperator(compoundPredicate, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 225676213fd8ed5..a303a17736d53a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -86,6 +86,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Cardinality; import org.apache.doris.nereids.trees.expressions.functions.scalar.Cbrt; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ceil; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Char; import org.apache.doris.nereids.trees.expressions.functions.scalar.CharacterLength; import org.apache.doris.nereids.trees.expressions.functions.scalar.Coalesce; import org.apache.doris.nereids.trees.expressions.functions.scalar.Concat; @@ -96,6 +97,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.ConvertTz; import org.apache.doris.nereids.trees.expressions.functions.scalar.Cos; import org.apache.doris.nereids.trees.expressions.functions.scalar.CountEqual; +import org.apache.doris.nereids.trees.expressions.functions.scalar.CreateMap; +import org.apache.doris.nereids.trees.expressions.functions.scalar.CreateNamedStruct; +import org.apache.doris.nereids.trees.expressions.functions.scalar.CreateStruct; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentCatalog; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentDate; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentTime; @@ -130,6 +134,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.E; import org.apache.doris.nereids.trees.expressions.functions.scalar.ElementAt; import org.apache.doris.nereids.trees.expressions.functions.scalar.Elt; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncryptKeyRef; import org.apache.doris.nereids.trees.expressions.functions.scalar.EndsWith; import org.apache.doris.nereids.trees.expressions.functions.scalar.EsQuery; import org.apache.doris.nereids.trees.expressions.functions.scalar.Exp; @@ -199,6 +204,11 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Lpad; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ltrim; import org.apache.doris.nereids.trees.expressions.functions.scalar.MakeDate; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapContainsKey; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapContainsValue; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapKeys; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapSize; +import org.apache.doris.nereids.trees.expressions.functions.scalar.MapValues; import org.apache.doris.nereids.trees.expressions.functions.scalar.Mask; import org.apache.doris.nereids.trees.expressions.functions.scalar.MaskFirstN; import org.apache.doris.nereids.trees.expressions.functions.scalar.MaskLastN; @@ -302,6 +312,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.StrLeft; import org.apache.doris.nereids.trees.expressions.functions.scalar.StrRight; import org.apache.doris.nereids.trees.expressions.functions.scalar.StrToDate; +import org.apache.doris.nereids.trees.expressions.functions.scalar.StructElement; import org.apache.doris.nereids.trees.expressions.functions.scalar.SubBitmap; import org.apache.doris.nereids.trees.expressions.functions.scalar.SubReplace; import org.apache.doris.nereids.trees.expressions.functions.scalar.Substring; @@ -623,6 +634,10 @@ default R visitConcat(Concat concat, C context) { return visitScalarFunction(concat, context); } + default R visitChar(Char charFunc, C context) { + return visitScalarFunction(charFunc, context); + } + default R visitConcatWs(ConcatWs concatWs, C context) { return visitScalarFunction(concatWs, context); } @@ -835,6 +850,10 @@ default R visitEndsWith(EndsWith endsWith, C context) { return visitScalarFunction(endsWith, context); } + default R visitEncryptKeyRef(EncryptKeyRef encryptKeyRef, C context) { + return visitScalarFunction(encryptKeyRef, context); + } + default R visitEsQuery(EsQuery esQuery, C context) { return visitScalarFunction(esQuery, context); } @@ -1658,4 +1677,44 @@ default R visitJavaUdf(JavaUdf javaUdf, C context) { default R visitAliasUdf(AliasUdf aliasUdf, C context) { return visitScalarFunction(aliasUdf, context); } + + // map functions + + default R visitCreateMap(CreateMap createMap, C context) { + return visitScalarFunction(createMap, context); + } + + default R visitMapContainsKey(MapContainsKey mapContainsKey, C context) { + return visitScalarFunction(mapContainsKey, context); + } + + default R visitMapContainsValue(MapContainsValue mapContainsValue, C context) { + return visitScalarFunction(mapContainsValue, context); + } + + default R visitMapKeys(MapKeys mapKeys, C context) { + return visitScalarFunction(mapKeys, context); + } + + default R visitMapSize(MapSize mapSize, C context) { + return visitScalarFunction(mapSize, context); + } + + default R visitMapValues(MapValues mapValues, C context) { + return visitScalarFunction(mapValues, context); + } + + // struct function + + default R visitCreateStruct(CreateStruct createStruct, C context) { + return visitScalarFunction(createStruct, context); + } + + default R visitCreateNamedStruct(CreateNamedStruct createNamedStruct, C context) { + return visitScalarFunction(createNamedStruct, context); + } + + default R visitStructElement(StructElement structElement, C context) { + return visitScalarFunction(structElement, context); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java index 0ca3e2936343008..97614321ce3d57a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java @@ -30,6 +30,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import java.util.List; import java.util.Optional; @@ -114,6 +115,18 @@ default List computeOutput() { throw new IllegalStateException("Not support compute output for " + getClass().getName()); } + /** + * Get the input relation ids set of the plan. + * @return The result is collected from all inputs relations + */ + default Set getInputRelations() { + Set relationIdSet = Sets.newHashSet(); + children().forEach( + plan -> relationIdSet.addAll(plan.getInputRelations()) + ); + return relationIdSet; + } + String treeString(); Plan withGroupExpression(Optional groupExpression); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index da98e0e4d14f4e4..b54b12f18caee11 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -120,5 +120,6 @@ public enum PlanType { EXPLAIN_COMMAND, INSERT_INTO_TABLE_COMMAND, SELECT_INTO_OUTFILE_COMMAND, - UPDATE_COMMAND + UPDATE_COMMAND, + EXPORT_COMMAND } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java index 43eec60d48ac075..dba753264ae218c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java @@ -53,17 +53,19 @@ public class DeleteCommand extends Command implements ForwardWithSync, Explainab private final List partitions; private LogicalPlan logicalQuery; private OlapTable targetTable; + private final Optional cte; /** * constructor */ public DeleteCommand(List nameParts, String tableAlias, List partitions, - LogicalPlan logicalQuery) { + LogicalPlan logicalQuery, Optional cte) { super(PlanType.DELETE_COMMAND); this.nameParts = Utils.copyRequiredList(nameParts); this.tableAlias = tableAlias; this.partitions = Utils.copyRequiredList(partitions); this.logicalQuery = logicalQuery; + this.cte = cte; } @Override @@ -110,13 +112,13 @@ public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuer } logicalQuery = new LogicalProject<>(selectLists, logicalQuery); - - boolean isPartialUpdate = false; - if (targetTable.getEnableUniqueKeyMergeOnWrite() - && cols.size() < targetTable.getColumns().size()) { - isPartialUpdate = true; + if (cte.isPresent()) { + logicalQuery = ((LogicalPlan) cte.get().withChildren(logicalQuery)); } + boolean isPartialUpdate = targetTable.getEnableUniqueKeyMergeOnWrite() + && cols.size() < targetTable.getColumns().size(); + // make UnboundTableSink return new UnboundOlapTableSink<>(nameParts, cols, ImmutableList.of(), partitions, isPartialUpdate, logicalQuery); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExportCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExportCommand.java new file mode 100644 index 000000000000000..dbfd00059dac4f2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExportCommand.java @@ -0,0 +1,87 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.BrokerDesc; +import org.apache.doris.analysis.ExportStmt; +import org.apache.doris.analysis.PartitionNames; +import org.apache.doris.analysis.TableName; +import org.apache.doris.analysis.TableRef; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import java.util.List; +import java.util.Map; + +/** + * export table + */ +public class ExportCommand extends Command implements ForwardWithSync { + private List nameParts; + private String whereSql; + private String path; + private List partitionsNameList; + private Map fileProperties; + private BrokerDesc brokerDesc; + + /** + * constructor of ExportCommand + */ + public ExportCommand(List nameParts, List partitions, String whereSql, String path, + Map fileProperties, BrokerDesc brokerDesc) { + super(PlanType.EXPORT_COMMAND); + this.nameParts = nameParts; + this.partitionsNameList = partitions; + this.whereSql = whereSql; + this.path = path.trim(); + this.fileProperties = fileProperties; + this.brokerDesc = brokerDesc; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + ExportStmt exportStmt = generateExportStmt(); + Analyzer analyzer = new Analyzer(ctx.getEnv(), ctx); + exportStmt.analyze(analyzer); + ctx.getEnv().getExportMgr().addExportJobAndRegisterTask(exportStmt); + } + + private ExportStmt generateExportStmt() { + // generate tableRef + PartitionNames partitionNames = null; + if (!this.partitionsNameList.isEmpty()) { + partitionNames = new PartitionNames(false, this.partitionsNameList); + } + TableRef tableRef = new TableRef(new TableName(getTableName()), null, partitionNames, null, null, null); + return new ExportStmt(tableRef, whereSql, path, fileProperties, brokerDesc); + } + + public String getTableName() { + return nameParts.stream().map(Utils::quoteIfNeeded) + .reduce((left, right) -> left + "." + right).orElse(""); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitExportCommand(this, context); + } +} 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 3b9a694c6aa3361..81a35928696f935 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 @@ -71,17 +71,19 @@ public class UpdateCommand extends Command implements ForwardWithSync, Explainab private final @Nullable String tableAlias; private final LogicalPlan logicalQuery; private OlapTable targetTable; + private final Optional cte; /** * constructor */ public UpdateCommand(List nameParts, @Nullable String tableAlias, List assignments, - LogicalPlan logicalQuery) { + LogicalPlan logicalQuery, Optional cte) { super(PlanType.UPDATE_COMMAND); this.nameParts = Utils.copyRequiredList(nameParts); this.assignments = Utils.copyRequiredList(assignments); this.tableAlias = tableAlias; this.logicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery is required in update command"); + this.cte = cte; } @Override @@ -117,6 +119,9 @@ public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuer } logicalQuery = new LogicalProject<>(selectItems, logicalQuery); + if (cte.isPresent()) { + logicalQuery = ((LogicalPlan) cte.get().withChildren(logicalQuery)); + } // make UnboundTableSink return new UnboundOlapTableSink<>(nameParts, ImmutableList.of(), ImmutableList.of(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java index 485111ff0e4c196..bafb1d429b68e21 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.logical; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.rules.exploration.join.JoinReorderContext; @@ -64,6 +65,8 @@ public class LogicalJoin hashJoinConjuncts, + List otherJoinConjuncts, + JoinHint hint, + Optional markJoinSlotReference, + LEFT_CHILD_TYPE leftChild, RIGHT_CHILD_TYPE rightChild) { + this(joinType, hashJoinConjuncts, + otherJoinConjuncts, hint, markJoinSlotReference, + Optional.empty(), Optional.empty(), leftChild, rightChild); + this.bitmap = LongBitmap.or(this.bitmap, bitmap); + } + public LogicalJoin( JoinType joinType, List hashJoinConjuncts, @@ -264,6 +281,14 @@ public Optional getMarkJoinSlotReference() { return markJoinSlotReference; } + public long getBitmap() { + return bitmap; + } + + public void setBitmap(long bitmap) { + this.bitmap = bitmap; + } + @Override public LEFT_CHILD_TYPE left() { return (LEFT_CHILD_TYPE) child(0); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java index 9945bb5606d8e99..7423b225765fcfd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java @@ -37,11 +37,13 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; /** @@ -298,6 +300,13 @@ public List computeOutput() { .collect(ImmutableList.toImmutableList()); } + @Override + public Set getInputRelations() { + Set relationIdSet = Sets.newHashSet(); + relationIdSet.add(relationId); + return relationIdSet; + } + /** * Get the slot under the index, * and create a new slotReference for the slot that has not appeared in the materialized view. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java index ba4e4c6d7ae9896..6e3ebf9d8f915e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java @@ -231,7 +231,6 @@ public JSONObject toJson() { } public LogicalProject readFromJson(JSONObject logicalProject) { - return new LogicalProject<>(ImmutableList.of(new UnboundStar(ImmutableList.of())), null, null, isDistinct); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRelation.java index 8b1a3e598794620..4a3f5005c88659b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRelation.java @@ -26,11 +26,13 @@ import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; import org.json.JSONObject; import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.Set; /** * Logical relation plan. @@ -89,4 +91,11 @@ public JSONObject toJson() { logicalRelation.put("Properties", properties); return logicalRelation; } + + @Override + public Set getInputRelations() { + Set relationIdSet = Sets.newHashSet(); + relationIdSet.add(relationId); + return relationIdSet; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRepeat.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRepeat.java index 6c4dc30631719b8..61fc8584d401636 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRepeat.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRepeat.java @@ -117,9 +117,7 @@ public List getExpressions() { .build(); } - /** - * Determine the equality with another plan - */ + @Override public boolean equals(Object o) { if (this == o) { return true; @@ -127,9 +125,8 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - LogicalRepeat that = (LogicalRepeat) o; - return Objects.equals(groupingSets, that.groupingSets) - && Objects.equals(outputExpressions, that.outputExpressions); + LogicalRepeat that = (LogicalRepeat) o; + return groupingSets.equals(that.groupingSets) && outputExpressions.equals(that.outputExpressions); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index fbdf1085774ea88..4b974f8bfcb50f5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; +import org.apache.doris.nereids.trees.plans.commands.ExportCommand; import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.UpdateCommand; @@ -49,4 +50,8 @@ default R visitUpdateCommand(UpdateCommand updateCommand, C context) { default R visitDeleteCommand(DeleteCommand deleteCommand, C context) { return visitCommand(deleteCommand, context); } + + default R visitExportCommand(ExportCommand exportCommand, C context) { + return visitCommand(exportCommand, context); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java index 43d09b67b440934..0e894f7d1add0fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java @@ -112,14 +112,12 @@ public static Literal promoteLiteral(Object value, DataType dataType) { public static DataType convertPrimitiveFromStrings(List types) { String type = types.get(0).toLowerCase().trim(); switch (type) { - case "bool": case "boolean": return BooleanType.INSTANCE; case "tinyint": return TinyIntType.INSTANCE; case "smallint": return SmallIntType.INSTANCE; - case "integer": case "int": return IntegerType.INSTANCE; case "bigint": @@ -163,7 +161,11 @@ public static DataType convertPrimitiveFromStrings(List types) { case 1: return VarcharType.SYSTEM_DEFAULT; case 2: - return VarcharType.createVarcharType(Integer.parseInt(types.get(1))); + if (types.get(1).equals("*")) { + return VarcharType.SYSTEM_DEFAULT; + } else { + return VarcharType.createVarcharType(Integer.parseInt(types.get(1))); + } default: throw new AnalysisException("Nereids do not support type: " + type); } @@ -173,7 +175,11 @@ public static DataType convertPrimitiveFromStrings(List types) { case 1: return CharType.SYSTEM_DEFAULT; case 2: - return CharType.createCharType(Integer.parseInt(types.get(1))); + if (types.get(1).equals("*")) { + return CharType.SYSTEM_DEFAULT; + } else { + return CharType.createCharType(Integer.parseInt(types.get(1))); + } default: throw new AnalysisException("Nereids do not support type: " + type); } @@ -211,6 +217,7 @@ public static DataType convertPrimitiveFromStrings(List types) { case "quantile_state": return QuantileStateType.INSTANCE; case "json": + case "jsonb": return JsonType.INSTANCE; default: throw new AnalysisException("Nereids do not support type: " + type); @@ -290,11 +297,14 @@ public static DataType fromCatalogType(Type type) { } else if (type.isJsonbType()) { return JsonType.INSTANCE; } else if (type.isStructType()) { - // TODO: support struct type really - return StructType.INSTANCE; + List structFields = ((org.apache.doris.catalog.StructType) (type)).getFields().stream() + .map(cf -> new StructField(cf.getName(), fromCatalogType(cf.getType()), + cf.getContainsNull(), cf.getComment())) + .collect(ImmutableList.toImmutableList()); + return new StructType(structFields); } else if (type.isMapType()) { - // TODO: support map type really - return MapType.INSTANCE; + org.apache.doris.catalog.MapType mapType = (org.apache.doris.catalog.MapType) type; + return MapType.of(fromCatalogType(mapType.getKeyType()), fromCatalogType(mapType.getValueType())); } else if (type.isArrayType()) { org.apache.doris.catalog.ArrayType arrayType = (org.apache.doris.catalog.ArrayType) type; return ArrayType.of(fromCatalogType(arrayType.getItemType()), arrayType.getContainsNull()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/MapType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/MapType.java index 111c1c82922a2c0..28d2ed67400ac38 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/MapType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/MapType.java @@ -28,8 +28,7 @@ @Developing public class MapType extends DataType { - // TODO: remove it? - public static final MapType INSTANCE = new MapType(); + public static final MapType SYSTEM_DEFAULT = new MapType(); public static final int WIDTH = 24; 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 new file mode 100644 index 000000000000000..d42d7f960ad3801 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructField.java @@ -0,0 +1,104 @@ +// 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.types; + +import org.apache.doris.nereids.util.Utils; + +import java.util.Objects; + +/** + * A field inside a StructType. + */ +public class StructField { + + private final String name; + private final DataType dataType; + private final boolean nullable; + private final String comment; + + /** + * StructField Constructor + * @param name The name of this field + * @param dataType The data type of this field + * @param nullable Indicates if values of this field can be `null` values + */ + public StructField(String name, DataType dataType, boolean nullable, String comment) { + this.name = Objects.requireNonNull(name, "name should not be null"); + this.dataType = Objects.requireNonNull(dataType, "dataType should not be null"); + this.nullable = nullable; + this.comment = Objects.requireNonNull(comment, "comment should not be null"); + } + + public String getName() { + return name; + } + + public DataType getDataType() { + return dataType; + } + + public boolean isNullable() { + return nullable; + } + + public String getComment() { + return comment; + } + + public StructField withDataType(DataType dataType) { + 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); + } + + public String toSql() { + return name + ":" + dataType.toSql() + + (nullable ? " " : " NOT NULL") + + (comment.isEmpty() ? "" : " COMMENT " + comment); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StructField that = (StructField) o; + return nullable == that.nullable && Objects.equals(name, that.name) && Objects.equals(dataType, + that.dataType); + } + + @Override + public int hashCode() { + return Objects.hash(name, dataType, nullable); + } + + @Override + public String toString() { + return Utils.toSqlString("StructField", + "name", name, + "dataType", dataType, + "nullable", nullable, + "comment", comment); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructType.java index 9359df15f7dde68..e4ca5e9db7e185a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructType.java @@ -19,12 +19,18 @@ import org.apache.doris.catalog.Type; import org.apache.doris.nereids.annotation.Developing; +import org.apache.doris.nereids.exceptions.AnalysisException; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSortedMap; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.function.Supplier; +import java.util.stream.Collectors; /** * Struct type in Nereids. @@ -32,28 +38,43 @@ @Developing public class StructType extends DataType { - public static final StructType INSTANCE = new StructType(); + public static final StructType SYSTEM_DEFAULT = new StructType(); public static final int WIDTH = 24; - private final Map items; + private final List fields; + private final Supplier> nameToFields; private StructType() { - items = ImmutableMap.of(); + nameToFields = Suppliers.memoize(ImmutableMap::of); + fields = ImmutableList.of(); } - public StructType(Map items) { - this.items = ImmutableSortedMap.copyOf(Objects.requireNonNull(items, "items should not be null"), - String.CASE_INSENSITIVE_ORDER); + /** + * construct struct type. + */ + public StructType(List fields) { + this.fields = ImmutableList.copyOf(Objects.requireNonNull(fields, "fields should not be null")); + this.nameToFields = Suppliers.memoize(() -> this.fields.stream().collect(ImmutableMap.toImmutableMap( + StructField::getName, f -> f, (f1, f2) -> { + throw new AnalysisException("The name of the struct field cannot be repeated." + + " same name fields are " + f1 + " and " + f2); + }))); } - public Map getItems() { - return items; + public List getFields() { + return fields; + } + + public Map getNameToFields() { + return nameToFields.get(); } @Override public Type toCatalogDataType() { - return Type.STRUCT; + return new org.apache.doris.catalog.StructType(fields.stream() + .map(StructField::toCatalogDataType) + .collect(Collectors.toCollection(ArrayList::new))); } @Override @@ -68,7 +89,22 @@ public String simpleString() { @Override public boolean equals(Object o) { - return o instanceof StructType; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + StructType that = (StructType) o; + return Objects.equals(fields, that.fields); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), fields); } @Override @@ -78,6 +114,11 @@ public int width() { @Override public String toSql() { - return "STRUCT"; + return "STRUCT<" + fields.stream().map(StructField::toSql).collect(Collectors.joining(", ")) + ">"; + } + + @Override + public String toString() { + return "STRUCT<" + fields.stream().map(StructField::toString).collect(Collectors.joining(", ")) + ">"; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java index 7c9a3e4c5f7d958..185c445647aabe9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java @@ -43,6 +43,8 @@ import org.apache.doris.nereids.trees.expressions.Subtract; import org.apache.doris.nereids.trees.expressions.TimestampArithmetic; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Array; +import org.apache.doris.nereids.trees.expressions.functions.scalar.CreateMap; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArray; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonObject; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -58,6 +60,7 @@ import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.expressions.literal.LargeIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.MapLiteral; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.literal.SmallIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; @@ -82,6 +85,7 @@ import org.apache.doris.nereids.types.NullType; import org.apache.doris.nereids.types.SmallIntType; import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.StructField; import org.apache.doris.nereids.types.StructType; import org.apache.doris.nereids.types.TimeType; import org.apache.doris.nereids.types.TimeV2Type; @@ -150,7 +154,22 @@ public static Optional implicitCast(DataType input, DataType expected) } return Optional.empty(); } else if (input instanceof StructType && expected instanceof StructType) { - throw new AnalysisException("not support struct type now."); + List inputFields = ((StructType) input).getFields(); + List expectedFields = ((StructType) expected).getFields(); + if (inputFields.size() != expectedFields.size()) { + return Optional.empty(); + } + List newFields = Lists.newArrayList(); + for (int i = 0; i < inputFields.size(); i++) { + Optional newDataType = implicitCast(inputFields.get(i).getDataType(), + expectedFields.get(i).getDataType()); + if (newDataType.isPresent()) { + newFields.add(inputFields.get(i).withDataType(newDataType.get())); + } else { + return Optional.empty(); + } + } + return Optional.of(new StructType(newFields)); } else { return implicitCastPrimitive(input, expected); } @@ -244,7 +263,7 @@ public static boolean hasCharacterType(DataType dataType) { return hasCharacterType(((MapType) dataType).getKeyType()) || hasCharacterType(((MapType) dataType).getValueType()); } else if (dataType instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + return ((StructType) dataType).getFields().stream().anyMatch(f -> hasCharacterType(f.getDataType())); } return dataType instanceof CharacterType; } @@ -290,7 +309,17 @@ private static boolean matchesType(DataType input, DataType target) { return matchesType(((MapType) input).getKeyType(), ((MapType) target).getKeyType()) && matchesType(((MapType) input).getValueType(), ((MapType) target).getValueType()); } else if (input instanceof StructType && target instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + List inputFields = ((StructType) input).getFields(); + List targetFields = ((StructType) target).getFields(); + if (inputFields.size() != targetFields.size()) { + return false; + } + for (int i = 0; i < inputFields.size(); i++) { + if (!matchesType(inputFields.get(i).getDataType(), targetFields.get(i).getDataType())) { + return false; + } + } + return true; } else { if (input instanceof NullType) { return false; @@ -496,11 +525,45 @@ public static Expression processBoundFunction(BoundFunction boundFunction) { if (boundFunction instanceof JsonArray || boundFunction instanceof JsonObject) { boundFunction = TypeCoercionUtils.fillJsonTypeArgument(boundFunction, boundFunction instanceof JsonObject); } + if (boundFunction instanceof CreateMap) { + return processCreateMap((CreateMap) boundFunction); + } // type coercion return implicitCastInputTypes(boundFunction, boundFunction.expectedInputTypes()); } + private static Expression processCreateMap(CreateMap createMap) { + if (createMap.arity() == 0) { + return new MapLiteral(); + } + List keys = Lists.newArrayList(); + List values = Lists.newArrayList(); + for (int i = 0; i < createMap.arity(); i++) { + if (i % 2 == 0) { + keys.add(createMap.child(i)); + } else { + values.add(createMap.child(i)); + } + } + // TODO: use the find common type to get key and value type after we redefine type coercion in Doris. + Array keyArray = new Array(keys.toArray(new Expression[0])); + Array valueArray = new Array(values.toArray(new Expression[0])); + keyArray = (Array) implicitCastInputTypes(keyArray, keyArray.expectedInputTypes()); + valueArray = (Array) implicitCastInputTypes(valueArray, valueArray.expectedInputTypes()); + DataType keyType = ((ArrayType) (keyArray.getDataType())).getItemType(); + DataType valueType = ((ArrayType) (valueArray.getDataType())).getItemType(); + ImmutableList.Builder newChildren = ImmutableList.builder(); + for (int i = 0; i < createMap.arity(); i++) { + if (i % 2 == 0) { + newChildren.add(castIfNotSameType(createMap.child(i), keyType)); + } else { + newChildren.add(castIfNotSameType(createMap.child(i), valueType)); + } + } + return createMap.withChildren(newChildren.build()); + } + /** * process divide */ @@ -957,7 +1020,22 @@ private static Optional findCommonComplexTypeForComparison( return Optional.of(MapType.of(keyType.get(), valueType.get())); } } else if (left instanceof StructType && right instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + List leftFields = ((StructType) left).getFields(); + List rightFields = ((StructType) right).getFields(); + if (leftFields.size() != rightFields.size()) { + return Optional.empty(); + } + List newFields = Lists.newArrayList(); + for (int i = 0; i < leftFields.size(); i++) { + Optional newDataType = findCommonComplexTypeForComparison(leftFields.get(i).getDataType(), + rightFields.get(i).getDataType(), intStringToString); + if (newDataType.isPresent()) { + newFields.add(leftFields.get(i).withDataType(newDataType.get())); + } else { + return Optional.empty(); + } + } + return Optional.of(new StructType(newFields)); } return Optional.empty(); } @@ -1155,7 +1233,22 @@ private static Optional findCommonComplexTypeForCaseWhen(DataType left return Optional.of(MapType.of(keyType.get(), valueType.get())); } } else if (left instanceof StructType && right instanceof StructType) { - throw new AnalysisException("do not support struct type now"); + List leftFields = ((StructType) left).getFields(); + List rightFields = ((StructType) right).getFields(); + if (leftFields.size() != rightFields.size()) { + return Optional.empty(); + } + List newFields = Lists.newArrayList(); + for (int i = 0; i < leftFields.size(); i++) { + Optional newDataType = findCommonComplexTypeForCaseWhen(leftFields.get(i).getDataType(), + rightFields.get(i).getDataType()); + if (newDataType.isPresent()) { + newFields.add(leftFields.get(i).withDataType(newDataType.get())); + } else { + return Optional.empty(); + } + } + return Optional.of(new StructType(newFields)); } return Optional.empty(); } 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 06486fc49cb778f..9f9baf276a490ee 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 @@ -555,6 +555,11 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { Env.getCurrentGlobalTransactionMgr().replayBatchRemoveTransactionV2(operation); break; } + case OperationType.OP_SET_TABLE_STATUS: { + final SetTableStatusOperationLog log = (SetTableStatusOperationLog) journal.getData(); + env.replaySetTableStatus(log); + break; + } case OperationType.OP_CREATE_REPOSITORY: { Repository repository = (Repository) journal.getData(); env.getBackupHandler().getRepoMgr().addAndInitRepoIfNotExist(repository, true); @@ -830,6 +835,11 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { env.getAlterInstance().replayModifyComment(operation); break; } + case OperationType.OP_SET_PARTITION_VERSION: { + SetPartitionVersionOperationLog log = (SetPartitionVersionOperationLog) journal.getData(); + env.replaySetPartitionVersion(log); + break; + } case OperationType.OP_ALTER_ROUTINE_LOAD_JOB: { AlterRoutineLoadJobOperationLog log = (AlterRoutineLoadJobOperationLog) journal.getData(); env.getRoutineLoadManager().replayAlterRoutineLoadJob(log); @@ -1734,6 +1744,10 @@ public void logAlterRoutineLoadJob(AlterRoutineLoadJobOperationLog log) { logEdit(OperationType.OP_ALTER_ROUTINE_LOAD_JOB, log); } + public void logSetPartitionVersion(SetPartitionVersionOperationLog log) { + logEdit(OperationType.OP_SET_PARTITION_VERSION, log); + } + public void logGlobalVariableV2(GlobalVarPersistInfo info) { logEdit(OperationType.OP_GLOBAL_VARIABLE_V2, info); } @@ -1746,6 +1760,10 @@ public void logBatchRemoveTransactions(BatchRemoveTransactionsOperationV2 op) { logEdit(OperationType.OP_BATCH_REMOVE_TXNS_V2, op); } + public void logSetTableStatus(SetTableStatusOperationLog log) { + logEdit(OperationType.OP_SET_TABLE_STATUS, log); + } + public void logModifyComment(ModifyCommentOperationLog op) { logEdit(OperationType.OP_MODIFY_COMMENT, op); } 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 56e4c7ad639005e..ccfa2831778b3d5 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 @@ -105,6 +105,7 @@ public class OperationType { public static final short OP_BACKEND_TABLETS_INFO = 46; public static final short OP_SET_REPLICA_STATUS = 47; public static final short OP_BACKEND_REPLICAS_INFO = 48; + public static final short OP_SET_PARTITION_VERSION = 49; public static final short OP_ADD_BACKEND = 50; public static final short OP_DROP_BACKEND = 51; @@ -225,6 +226,9 @@ public class OperationType { // set table default distribution bucket num public static final short OP_MODIFY_DISTRIBUTION_BUCKET_NUM = 268; + // set table status + public static final short OP_SET_TABLE_STATUS = 269; + // plugin 270~275 public static final short OP_INSTALL_PLUGIN = 270; diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/SetPartitionVersionOperationLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/SetPartitionVersionOperationLog.java new file mode 100644 index 000000000000000..ae5b3a4c382fbc8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/SetPartitionVersionOperationLog.java @@ -0,0 +1,63 @@ +// 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.persist; + +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; +import lombok.Getter; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +@Getter +public class SetPartitionVersionOperationLog implements Writable { + + @SerializedName(value = "database") + private String database; + + @SerializedName(value = "table") + private String table; + + @SerializedName(value = "partitionId") + private long partitionId; + + @SerializedName(value = "visibleVersion") + private long visibleVersion; + + public SetPartitionVersionOperationLog(String database, String table, long partitionId, long visibleVersion) { + this.database = database; + this.table = table; + this.partitionId = partitionId; + this.visibleVersion = visibleVersion; + } + + public static SetPartitionVersionOperationLog read(DataInput in) throws IOException { + String json = Text.readString(in); + return GsonUtils.GSON.fromJson(json, SetPartitionVersionOperationLog.class); + } + + @Override + public void write(DataOutput out) throws IOException { + String json = GsonUtils.GSON.toJson(this); + Text.writeString(out, json); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/SetTableStatusOperationLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/SetTableStatusOperationLog.java new file mode 100644 index 000000000000000..367fafaaba67929 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/SetTableStatusOperationLog.java @@ -0,0 +1,65 @@ +// 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.persist; + +import org.apache.doris.catalog.OlapTable.OlapTableState; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; +import lombok.Getter; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public class SetTableStatusOperationLog implements Writable { + @SerializedName(value = "db") + private String db; + @SerializedName(value = "table") + private String table; + @Getter + @SerializedName(value = "state") + private OlapTableState state; + + public SetTableStatusOperationLog(String db, String table, OlapTableState state) { + this.db = db; + this.table = table; + this.state = state; + } + + public String getDbName() { + return db; + } + + public String getTblName() { + return table; + } + + public static SetTableStatusOperationLog read(DataInput in) throws IOException { + String json = Text.readString(in); + return GsonUtils.GSON.fromJson(json, SetTableStatusOperationLog.class); + } + + @Override + public void write(DataOutput out) throws IOException { + String json = GsonUtils.GSON.toJson(this); + Text.writeString(out, json); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index 196e68bad3572fa..aa55936a0519167 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java @@ -44,6 +44,8 @@ import org.apache.doris.catalog.SparkResource; import org.apache.doris.catalog.StructType; import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.external.DeltaLakeExternalDataBase; +import org.apache.doris.catalog.external.DeltaLakeExternalTable; import org.apache.doris.catalog.external.EsExternalDatabase; import org.apache.doris.catalog.external.EsExternalTable; import org.apache.doris.catalog.external.ExternalDatabase; @@ -64,6 +66,7 @@ import org.apache.doris.datasource.HMSExternalCatalog; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.MaxComputeExternalCatalog; +import org.apache.doris.datasource.deltalake.DeltaLakeExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergDLFExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergGlueExternalCatalog; @@ -209,7 +212,8 @@ public class GsonUtils { .registerSubtype(IcebergHadoopExternalCatalog.class, IcebergHadoopExternalCatalog.class.getSimpleName()) .registerSubtype(PaimonExternalCatalog.class, PaimonExternalCatalog.class.getSimpleName()) .registerSubtype(PaimonHMSExternalCatalog.class, PaimonHMSExternalCatalog.class.getSimpleName()) - .registerSubtype(MaxComputeExternalCatalog.class, MaxComputeExternalCatalog.class.getSimpleName()); + .registerSubtype(MaxComputeExternalCatalog.class, MaxComputeExternalCatalog.class.getSimpleName()) + .registerSubtype(DeltaLakeExternalCatalog.class, DeltaLakeExternalCatalog.class.getSimpleName()); // routine load data source private static RuntimeTypeAdapterFactory rdsTypeAdapterFactory = RuntimeTypeAdapterFactory.of( @@ -228,7 +232,8 @@ public class GsonUtils { .registerSubtype(JdbcExternalDatabase.class, JdbcExternalDatabase.class.getSimpleName()) .registerSubtype(IcebergExternalDatabase.class, IcebergExternalDatabase.class.getSimpleName()) .registerSubtype(PaimonExternalDatabase.class, PaimonExternalDatabase.class.getSimpleName()) - .registerSubtype(MaxComputeExternalDatabase.class, MaxComputeExternalDatabase.class.getSimpleName()); + .registerSubtype(MaxComputeExternalDatabase.class, MaxComputeExternalDatabase.class.getSimpleName()) + .registerSubtype(DeltaLakeExternalDataBase.class, DeltaLakeExternalDataBase.class.getSimpleName()); private static RuntimeTypeAdapterFactory tblTypeAdapterFactory = RuntimeTypeAdapterFactory.of( TableIf.class, "clazz").registerSubtype(ExternalTable.class, ExternalTable.class.getSimpleName()) @@ -238,7 +243,8 @@ public class GsonUtils { .registerSubtype(JdbcExternalTable.class, JdbcExternalTable.class.getSimpleName()) .registerSubtype(IcebergExternalTable.class, IcebergExternalTable.class.getSimpleName()) .registerSubtype(PaimonExternalTable.class, PaimonExternalTable.class.getSimpleName()) - .registerSubtype(MaxComputeExternalTable.class, MaxComputeExternalTable.class.getSimpleName()); + .registerSubtype(MaxComputeExternalTable.class, MaxComputeExternalTable.class.getSimpleName()) + .registerSubtype(DeltaLakeExternalTable.class, DeltaLakeExternalTable.class.getSimpleName()); // runtime adapter for class "PartitionInfo" private static RuntimeTypeAdapterFactory partitionInfoTypeAdapterFactory diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OriginalPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OriginalPlanner.java index f77aba25dac1258..92047ea952b9959 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OriginalPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OriginalPlanner.java @@ -143,6 +143,13 @@ public void appendTupleInfo(StringBuilder str) { str.append(plannerContext.getRootAnalyzer().getDescTbl().getExplainString()); } + /** + * Return hint information. + */ + @Override + public void appendHintInfo(StringBuilder str) { + } + /** * Create plan fragments for an analyzed statement, given a set of execution options. The fragments are returned in * a list such that element i of that list can only consume output of the following fragments j > i. diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java index 48f2ba7cbd23a26..46c7bc91d55d5b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java @@ -85,6 +85,7 @@ public String getExplainString(ExplainOptions explainOptions) { if (explainLevel == org.apache.doris.thrift.TExplainLevel.VERBOSE) { appendTupleInfo(str); } + appendHintInfo(str); return str.toString(); } @@ -104,6 +105,8 @@ protected void handleLiteralInFe(LiteralExpr literalExpr, List data) { public void appendTupleInfo(StringBuilder stringBuilder) {} + public void appendHintInfo(StringBuilder stringBuilder) {} + public List getFragments() { return fragments; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java index faeb6183c009be3..08f883a6c86cb0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java @@ -2020,7 +2020,7 @@ private PlanNode createScanNode(Analyzer analyzer, TableRef tblRef, SelectStmt s scanNode = new HiveScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); break; default: - throw new UserException("Not supported table type" + table.getType()); + throw new UserException("Not supported table type: " + ((HMSExternalTable) table).getDlaType()); } break; case ICEBERG_EXTERNAL_TABLE: @@ -2044,7 +2044,7 @@ private PlanNode createScanNode(Analyzer analyzer, TableRef tblRef, SelectStmt s scanNode = new TestExternalTableScanNode(ctx.getNextNodeId(), tblRef.getDesc()); break; default: - throw new UserException("Not supported table type" + tblRef.getTable().getType()); + throw new UserException("Not supported table type: " + tblRef.getTable().getType()); } if (scanNode instanceof OlapScanNode || scanNode instanceof EsScanNode || scanNode instanceof FileQueryScanNode) { @@ -2762,7 +2762,7 @@ private List getPredicatesBoundedByGroupbysSourceExpr( while (sourceExpr instanceof SlotRef) { SlotRef slotRef = (SlotRef) sourceExpr; SlotDescriptor slotDesc = slotRef.getDesc(); - if (slotDesc.getSourceExprs().isEmpty()) { + if (slotDesc.getSourceExprs().size() != 1) { break; } sourceExpr = slotDesc.getSourceExprs().get(0); 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 40dba10c98a735e..e654a64c70759fa 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 @@ -257,7 +257,7 @@ public void createScanRangeLocations() throws UserException { ConnectContext.get().getExecutor().getSummaryProfile().setGetSplitsFinishTime(); } this.inputSplitsNum = inputSplits.size(); - if (inputSplits.isEmpty()) { + if (inputSplits.isEmpty() && !(getLocationType() == TFileType.FILE_STREAM)) { return; } TFileFormatType fileFormatType = getFileFormatType(); @@ -265,6 +265,27 @@ public void createScanRangeLocations() throws UserException { boolean isCsvOrJson = Util.isCsvFormat(fileFormatType) || fileFormatType == TFileFormatType.FORMAT_JSON; if (isCsvOrJson) { params.setFileAttributes(getFileAttributes()); + if (getLocationType() == TFileType.FILE_STREAM) { + params.setFileType(TFileType.FILE_STREAM); + params.setCompressType(TFileCompressType.PLAIN); + + TScanRangeLocations curLocations = newLocations(); + TFileRangeDesc rangeDesc = new TFileRangeDesc(); + rangeDesc.setLoadId(ConnectContext.get().queryId()); + rangeDesc.setSize(-1); + rangeDesc.setFileSize(-1); + curLocations.getScanRange().getExtScanRange().getFileScanRange().addToRanges(rangeDesc); + curLocations.getScanRange().getExtScanRange().getFileScanRange().setParams(params); + + TScanRangeLocation location = new TScanRangeLocation(); + long backendId = ConnectContext.get().getBackendId(); + Backend backend = Env.getCurrentSystemInfo().getIdToBackend().get(backendId); + location.setBackendId(backendId); + location.setServer(new TNetworkAddress(backend.getHost(), backend.getBePort())); + curLocations.addToLocations(location); + scanRangeLocations.add(curLocations); + return; + } } Map locationProperties = getLocationProperties(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/TVFScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/TVFScanNode.java index c069aa43c30d150..59f855f445a8bf2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/TVFScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/TVFScanNode.java @@ -129,6 +129,9 @@ public TableIf getTargetTable() { @Override public List getSplits() throws UserException { List splits = Lists.newArrayList(); + if (tableValuedFunction.getTFileType() == TFileType.FILE_STREAM) { + return splits; + } List fileStatuses = tableValuedFunction.getFileStatuses(); for (TBrokerFileStatus fileStatus : fileStatuses) { Path path = new Path(fileStatus.getPath()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/policy/PolicyMgr.java b/fe/fe-core/src/main/java/org/apache/doris/policy/PolicyMgr.java index 96004adf669b119..0ee44ba886d31d8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/policy/PolicyMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/policy/PolicyMgr.java @@ -533,4 +533,15 @@ public void checkStoragePolicyExist(String storagePolicyName) throws DdlExceptio readUnlock(); } } + + public boolean checkStoragePolicyIfSameResource(String policyName, String anotherPolicyName) { + Optional policy = findPolicy(policyName, PolicyTypeEnum.STORAGE); + Optional policy1 = findPolicy(anotherPolicyName, PolicyTypeEnum.STORAGE); + if (policy1.isPresent() && policy.isPresent()) { + StoragePolicy storagePolicy = (StoragePolicy) policy.get(); + StoragePolicy storagePolicy1 = (StoragePolicy) policy1.get(); + return storagePolicy1.getStorageResource().equals(storagePolicy.getStorageResource()); + } + return false; + } } 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 3e6ea543509ee8a..fd63385a33ba28f 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 @@ -76,7 +76,7 @@ public class ConnectContext { protected volatile long stmtId; protected volatile long forwardedStmtId; - // set for stream load with sql + // set for http_stream protected volatile TUniqueId loadId; protected volatile long backendId; protected volatile LoadTaskInfo streamLoadInfo; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 8c9f3fb09e28b96..1c97dadf3c7f620 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -72,6 +72,7 @@ import org.apache.doris.qe.QueryStatisticsItem.FragmentInstanceInfo; import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.rpc.RpcException; +import org.apache.doris.service.ExecuteEnv; import org.apache.doris.service.FrontendOptions; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; @@ -399,6 +400,7 @@ private void initQueryOptions(ConnectContext context) { this.queryOptions.setQueryTimeout(context.getExecTimeout()); this.queryOptions.setExecutionTimeout(context.getExecTimeout()); this.queryOptions.setEnableScanNodeRunSerial(context.getSessionVariable().isEnableScanRunSerial()); + this.queryOptions.setFeProcessUuid(ExecuteEnv.getInstance().getProcessUUID()); } public ConnectContext getConnectContext() { @@ -562,6 +564,33 @@ private void traceInstance() { } } + private void processFragmentAssignmentAndParams() throws Exception { + // prepare information + prepare(); + // compute Fragment Instance + computeScanRangeAssignment(); + + computeFragmentExecParams(); + } + + + public TExecPlanFragmentParams getStreamLoadPlan() throws Exception { + processFragmentAssignmentAndParams(); + + // This is a load process. + List relatedBackendIds = Lists.newArrayList(addressToBackendID.values()); + Env.getCurrentEnv().getLoadManager().initJobProgress(jobId, queryId, instanceIds, + relatedBackendIds); + Env.getCurrentEnv().getProgressManager().addTotalScanNums(String.valueOf(jobId), scanRangeNum); + LOG.info("dispatch load job: {} to {}", DebugUtil.printId(queryId), addressToBackendID.keySet()); + + executionProfile.markInstances(instanceIds); + List tExecPlanFragmentParams + = ((FragmentExecParams) this.fragmentExecParamsMap.values().toArray()[0]).toThrift(0); + TExecPlanFragmentParams fragmentParams = tExecPlanFragmentParams.get(0); + return fragmentParams; + } + // Initiate asynchronous execution of query. Returns as soon as all plan fragments // have started executing at their respective backends. // 'Request' must contain at least a coordinator plan fragment (ie, can't @@ -578,12 +607,7 @@ public void exec() throws Exception { DebugUtil.printId(queryId), fragments.get(0).toThrift()); } - // prepare information - prepare(); - // compute Fragment Instance - computeScanRangeAssignment(); - - computeFragmentExecParams(); + processFragmentAssignmentAndParams(); traceInstance(); 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 1d2d31f41631036..30863adf593da21 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 @@ -25,7 +25,9 @@ import org.apache.doris.analysis.AdminRebalanceDiskStmt; import org.apache.doris.analysis.AdminRepairTableStmt; import org.apache.doris.analysis.AdminSetConfigStmt; +import org.apache.doris.analysis.AdminSetPartitionVersionStmt; import org.apache.doris.analysis.AdminSetReplicaStatusStmt; +import org.apache.doris.analysis.AdminSetTableStatusStmt; import org.apache.doris.analysis.AlterCatalogNameStmt; import org.apache.doris.analysis.AlterCatalogPropertyStmt; import org.apache.doris.analysis.AlterColumnStatsStmt; @@ -255,6 +257,8 @@ public static void execute(Env env, DdlStmt ddlStmt) throws Exception { env.compactTable((AdminCompactTableStmt) ddlStmt); } else if (ddlStmt instanceof AdminSetConfigStmt) { env.setConfig((AdminSetConfigStmt) ddlStmt); + } else if (ddlStmt instanceof AdminSetTableStatusStmt) { + env.setTableStatus((AdminSetTableStatusStmt) ddlStmt); } else if (ddlStmt instanceof CreateFileStmt) { env.getSmallFileMgr().createFile((CreateFileStmt) ddlStmt); } else if (ddlStmt instanceof DropFileStmt) { @@ -267,6 +271,8 @@ public static void execute(Env env, DdlStmt ddlStmt) throws Exception { env.checkTablets((AdminCheckTabletsStmt) ddlStmt); } else if (ddlStmt instanceof AdminSetReplicaStatusStmt) { env.setReplicaStatus((AdminSetReplicaStatusStmt) ddlStmt); + } else if (ddlStmt instanceof AdminSetPartitionVersionStmt) { + env.setPartitionVersion((AdminSetPartitionVersionStmt) ddlStmt); } else if (ddlStmt instanceof CreateResourceStmt) { env.getResourceMgr().createResource((CreateResourceStmt) ddlStmt); } else if (ddlStmt instanceof DropResourceStmt) { 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 ee66fa3eb6eae71..f7afc5e33575d14 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 @@ -66,6 +66,7 @@ public class SessionVariable implements Serializable, Writable { public static final String EXEC_MEM_LIMIT = "exec_mem_limit"; public static final String SCAN_QUEUE_MEM_LIMIT = "scan_queue_mem_limit"; public static final String QUERY_TIMEOUT = "query_timeout"; + public static final String MAX_EXECUTION_TIME = "max_execution_time"; public static final String INSERT_TIMEOUT = "insert_timeout"; public static final String ENABLE_PROFILE = "enable_profile"; @@ -133,6 +134,9 @@ public class SessionVariable implements Serializable, Writable { public static final String DEFAULT_STORAGE_ENGINE = "default_storage_engine"; public static final String DEFAULT_TMP_STORAGE_ENGINE = "default_tmp_storage_engine"; + // Compatible with mysql + public static final String PROFILLING = "profiling"; + public static final String DIV_PRECISION_INCREMENT = "div_precision_increment"; // see comment of `doris_max_scan_key_num` and `max_pushdown_conditions_per_column` in BE config @@ -381,6 +385,8 @@ public class SessionVariable implements Serializable, Writable { public static final String ROUND_PRECISE_DECIMALV2_VALUE = "round_precise_decimalv2_value"; + public static final String ENABLE_DELETE_SUB_PREDICATE_V2 = "enable_delete_sub_predicate_v2"; + public static final String JDBC_CLICKHOUSE_QUERY_FINAL = "jdbc_clickhouse_query_final"; public static final String ENABLE_MEMTABLE_ON_SINK_NODE = @@ -724,6 +730,9 @@ public int getBeNumberForTest() { return beNumberForTest; } + @VariableMgr.VarAttr(name = PROFILLING) + public boolean profiling = false; + public void setBeNumberForTest(int beNumberForTest) { this.beNumberForTest = beNumberForTest; } @@ -1109,6 +1118,9 @@ public void setMaxJoinNumberOfReorder(int maxJoinNumberOfReorder) { @VariableMgr.VarAttr(name = PARALLEL_SYNC_ANALYZE_TASK_NUM) public int parallelSyncAnalyzeTaskNum = 2; + @VariableMgr.VarAttr(name = ENABLE_DELETE_SUB_PREDICATE_V2, fuzzy = true, needForward = true) + public boolean enableDeleteSubPredicateV2 = true; + @VariableMgr.VarAttr(name = TRUNCATE_CHAR_OR_VARCHAR_COLUMNS, description = {"是否按照表的 schema 来截断 char 或者 varchar 列。默认为 false。\n" + "因为外表会存在表的 schema 中 char 或者 varchar 列的最大长度和底层 parquet 或者 orc 文件中的 schema 不一致" @@ -1144,9 +1156,11 @@ public void initFuzzyModeVariables() { if (randomInt % 2 == 0) { this.rewriteOrToInPredicateThreshold = 100000; this.enableFunctionPushdown = false; + this.enableDeleteSubPredicateV2 = false; } else { this.rewriteOrToInPredicateThreshold = 2; this.enableFunctionPushdown = true; + this.enableDeleteSubPredicateV2 = true; } this.runtimeFilterType = 1 << randomInt; /* @@ -1311,6 +1325,7 @@ public int getInsertTimeoutS() { return insertTimeoutS; } + public void setInsertTimeoutS(int insertTimeoutS) { this.insertTimeoutS = insertTimeoutS; } @@ -2205,6 +2220,7 @@ public TQueryOptions toThrift() { tResult.setEnableParquetLazyMat(enableParquetLazyMat); tResult.setEnableOrcLazyMat(enableOrcLazyMat); + tResult.setEnableDeleteSubPredicateV2(enableDeleteSubPredicateV2); tResult.setTruncateCharOrVarcharColumns(truncateCharOrVarcharColumns); tResult.setEnableMemtableOnSinkNode(enableMemtableOnSinkNode); 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 f505089cd756c5c..5faf28b294d0b7a 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 @@ -2670,7 +2670,12 @@ private void handleShowAnalyze() { LocalDateTime.ofInstant(Instant.ofEpochMilli(analysisInfo.lastExecTimeInMs), ZoneId.systemDefault()))); row.add(analysisInfo.state.toString()); - row.add(Env.getCurrentEnv().getAnalysisManager().getJobProgress(analysisInfo.jobId)); + try { + row.add(Env.getCurrentEnv().getAnalysisManager().getJobProgress(analysisInfo.jobId)); + } catch (Exception e) { + row.add("N/A"); + LOG.warn("Failed to get progress for job: {}", analysisInfo, e); + } row.add(analysisInfo.scheduleType.toString()); resultRows.add(row); } 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 85235ec1592d344..8f654dc2924d0c1 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 @@ -2438,6 +2438,7 @@ public StatementBase setParsedStmt(StatementBase parsedStmt) { } public List executeInternalQuery() { + LOG.debug("INTERNAL QUERY: " + originStmt.toString()); try { List resultRows = new ArrayList<>(); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java index 980eb2d8293d054..ab15096f0cc2629 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java @@ -141,12 +141,16 @@ public class CacheTable implements Comparable { public long latestPartitionId; public long latestVersion; public long latestTime; + public long partitionNum; + public long sumOfPartitionNum; public CacheTable() { olapTable = null; latestPartitionId = 0; latestVersion = 0; latestTime = 0; + partitionNum = 0; + sumOfPartitionNum = 0; } @Override @@ -155,8 +159,8 @@ public int compareTo(CacheTable table) { } public void debug() { - LOG.debug("table {}, partition id {}, ver {}, time {}", olapTable.getName(), - latestPartitionId, latestVersion, latestTime); + LOG.debug("table {}, partition id {}, ver {}, time {}, partition num {}, sumOfPartitionNum: {}", + olapTable.getName(), latestPartitionId, latestVersion, latestTime, partitionNum, sumOfPartitionNum); } } @@ -216,7 +220,7 @@ private CacheMode innerCheckCacheMode(long now) { } if (enablePartitionCache() && ((OlapScanNode) node).getSelectedPartitionNum() > 1 && selectStmt.hasGroupByClause()) { - LOG.debug("more than one partition scanned when qeury has agg, partition cache cannot use, queryid {}", + LOG.debug("more than one partition scanned when query has agg, partition cache cannot use, queryid {}", DebugUtil.printId(queryId)); return CacheMode.None; } @@ -226,6 +230,7 @@ private CacheMode innerCheckCacheMode(long now) { MetricRepo.COUNTER_QUERY_OLAP_TABLE.increase(1L); Collections.sort(tblTimeList); latestTable = tblTimeList.get(0); + latestTable.sumOfPartitionNum = tblTimeList.stream().mapToLong(item -> item.partitionNum).sum(); latestTable.debug(); addAllViewStmt(selectStmt); @@ -328,6 +333,7 @@ private CacheMode innerCheckCacheModeSetOperation(long now) { MetricRepo.COUNTER_QUERY_OLAP_TABLE.increase(1L); Collections.sort(tblTimeList); latestTable = tblTimeList.get(0); + latestTable.sumOfPartitionNum = tblTimeList.stream().mapToLong(item -> item.partitionNum).sum(); latestTable.debug(); addAllViewStmt((SetOperationStmt) parsedStmt); @@ -382,6 +388,7 @@ private CacheMode innerCheckCacheModeForNereids(long now) { MetricRepo.COUNTER_QUERY_OLAP_TABLE.increase(1L); Collections.sort(tblTimeList); latestTable = tblTimeList.get(0); + latestTable.sumOfPartitionNum = tblTimeList.stream().mapToLong(item -> item.partitionNum).sum(); latestTable.debug(); if (((LogicalPlanAdapter) parsedStmt).getStatementContext().getParsedStatement().isExplain()) { @@ -576,6 +583,7 @@ private CacheTable getSelectedPartitionLastUpdateTime(OlapScanNode node) { CacheTable cacheTable = new CacheTable(); OlapTable olapTable = node.getOlapTable(); cacheTable.olapTable = olapTable; + cacheTable.partitionNum = node.getSelectedPartitionIds().size(); for (Long partitionId : node.getSelectedPartitionIds()) { Partition partition = olapTable.getPartition(partitionId); if (partition.getVisibleVersionTime() >= cacheTable.latestTime) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/RowBatchBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/RowBatchBuilder.java index 2cae79597a38b0f..665f47a793b1c2d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/RowBatchBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/RowBatchBuilder.java @@ -112,7 +112,7 @@ public void clear() { } public InternalService.PUpdateCacheRequest buildSqlUpdateRequest( - String sql, long partitionKey, long lastVersion, long lastestTime) { + String sql, long partitionKey, long lastVersion, long lastestTime, long partitionNum) { if (updateRequest == null) { updateRequest = InternalService.PUpdateCacheRequest.newBuilder() .setSqlKey(CacheProxy.getMd5(sql)) @@ -124,6 +124,7 @@ public InternalService.PUpdateCacheRequest buildSqlUpdateRequest( .setPartitionKey(partitionKey) .setLastVersion(lastVersion) .setLastVersionTime(lastestTime) + .setPartitionNum(partitionNum) .build()).setDataSize(dataSize).addAllRows( rowList.stream().map(row -> ByteString.copyFrom(row)) .collect(Collectors.toList()))).build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java index 04f0ed35cbde6fe..9135b453c657b72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java @@ -55,13 +55,18 @@ public String getSqlWithViewStmt() { return cacheKey; } + public long getSumOfPartitionNum() { + return latestTable.sumOfPartitionNum; + } + public InternalService.PFetchCacheResult getCacheData(Status status) { InternalService.PFetchCacheRequest request = InternalService.PFetchCacheRequest.newBuilder() .setSqlKey(CacheProxy.getMd5(getSqlWithViewStmt())) .addParams(InternalService.PCacheParam.newBuilder() .setPartitionKey(latestTable.latestPartitionId) .setLastVersion(latestTable.latestVersion) - .setLastVersionTime(latestTable.latestTime)) + .setLastVersionTime(latestTable.latestTime) + .setPartitionNum(latestTable.sumOfPartitionNum)) .build(); InternalService.PFetchCacheResult cacheResult = proxy.fetchCache(request, 10000, status); @@ -94,7 +99,7 @@ public void updateCache() { InternalService.PUpdateCacheRequest updateRequest = rowBatchBuilder.buildSqlUpdateRequest(getSqlWithViewStmt(), latestTable.latestPartitionId, - latestTable.latestVersion, latestTable.latestTime); + latestTable.latestVersion, latestTable.latestTime, latestTable.sumOfPartitionNum); if (updateRequest.getValuesCount() > 0) { CacheBeProxy proxy = new CacheBeProxy(); Status status = new Status(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FEFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FEFunctions.java index bce2dd7dfac2012..dd1bce88cc8ea2c 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FEFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FEFunctions.java @@ -26,6 +26,7 @@ import org.apache.doris.analysis.LiteralExpr; import org.apache.doris.analysis.NullLiteral; import org.apache.doris.analysis.StringLiteral; +import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.InvalidFormatException; @@ -111,37 +112,37 @@ public static DateLiteral addDate(LiteralExpr date, LiteralExpr day) throws Anal @FEFunction(name = "years_add", argTypes = { "DATETIME", "INT" }, returnType = "DATETIME") public static DateLiteral yearsAdd(LiteralExpr date, LiteralExpr year) throws AnalysisException { DateLiteral dateLiteral = (DateLiteral) date; - return dateLiteral.plusYears((int) year.getLongValue()); + return dateLiteral.plusYears(year.getLongValue()); } @FEFunction(name = "months_add", argTypes = { "DATETIME", "INT" }, returnType = "DATETIME") public static DateLiteral monthsAdd(LiteralExpr date, LiteralExpr month) throws AnalysisException { DateLiteral dateLiteral = (DateLiteral) date; - return dateLiteral.plusMonths((int) month.getLongValue()); + return dateLiteral.plusMonths(month.getLongValue()); } @FEFunction(name = "days_add", argTypes = { "DATETIME", "INT" }, returnType = "DATETIME") public static DateLiteral daysAdd(LiteralExpr date, LiteralExpr day) throws AnalysisException { DateLiteral dateLiteral = (DateLiteral) date; - return dateLiteral.plusDays((int) day.getLongValue()); + return dateLiteral.plusDays(day.getLongValue()); } @FEFunction(name = "hours_add", argTypes = { "DATETIME", "INT" }, returnType = "DATETIME") public static DateLiteral hoursAdd(LiteralExpr date, LiteralExpr hour) throws AnalysisException { DateLiteral dateLiteral = (DateLiteral) date; - return dateLiteral.plusHours((int) hour.getLongValue()); + return dateLiteral.plusHours(hour.getLongValue()); } @FEFunction(name = "minutes_add", argTypes = { "DATETIME", "INT" }, returnType = "DATETIME") public static DateLiteral minutesAdd(LiteralExpr date, LiteralExpr minute) throws AnalysisException { DateLiteral dateLiteral = (DateLiteral) date; - return dateLiteral.plusMinutes((int) minute.getLongValue()); + return dateLiteral.plusMinutes(minute.getLongValue()); } @FEFunction(name = "seconds_add", argTypes = { "DATETIME", "INT" }, returnType = "DATETIME") public static DateLiteral secondsAdd(LiteralExpr date, LiteralExpr second) throws AnalysisException { DateLiteral dateLiteral = (DateLiteral) date; - return dateLiteral.plusSeconds((int) second.getLongValue()); + return dateLiteral.plusSeconds(second.getLongValue()); } @FEFunction(name = "date_format", argTypes = { "DATETIME", "VARCHAR" }, returnType = "VARCHAR") @@ -350,6 +351,177 @@ public static DateLiteral toMondayV2(LiteralExpr arg) { return null; } + @FEFunction(name = "second_floor", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral second_floor(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, false, TimeUnit.SECOND); + } + + @FEFunction(name = "second_ceil", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral second_ceil(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, true, TimeUnit.SECOND); + } + + @FEFunction(name = "minute_floor", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral minute_floor(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, false, TimeUnit.MINUTE); + } + + @FEFunction(name = "minute_ceil", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral minute_ceil(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, true, TimeUnit.MINUTE); + } + + @FEFunction(name = "hour_floor", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral hour_floor(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, false, TimeUnit.HOUR); + } + + @FEFunction(name = "hour_ceil", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral hour_ceil(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, true, TimeUnit.HOUR); + } + + @FEFunction(name = "day_floor", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral day_floor(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, false, TimeUnit.DAY); + } + + @FEFunction(name = "day_ceil", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral day_ceil(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, true, TimeUnit.DAY); + } + + // get it's from be/src/vec/functions/function_datetime_floor_ceil.cpp##time_round + public static DateLiteral getFloorCeilDateLiteral(LiteralExpr datetime, LiteralExpr period, + LiteralExpr defaultDatetime, boolean isCeil, TimeUnit type) throws AnalysisException { + DateLiteral dt = ((DateLiteral) datetime); + DateLiteral start = ((DateLiteral) defaultDatetime); + long periodValue = ((IntLiteral) period).getValue(); + long diff = 0; + long trivialPart = 0; + + switch (type) { + case YEAR: { + diff = dt.getYear() - start.getYear(); + trivialPart = (dt.getLongValue() % 10000000000L) - (start.getLongValue() % 10000000000L); + break; + } + case MONTH: { + diff = (dt.getYear() - start.getYear()) * 12 + (dt.getMonth() - start.getMonth()); + trivialPart = (dt.getLongValue() % 100000000L) - (start.getLongValue() % 100000000L); + break; + } + case WEEK: { + diff = (dt.daynr() / 7) - (start.daynr() / 7); + long part2 = (dt.daynr() % 7) * 24 * 3600 + dt.getHour() * 3600 + dt.getMinute() * 60 + dt.getSecond(); + long part1 = (start.daynr() % 7) * 24 * 3600 + start.getHour() * 3600 + start.getMinute() * 60 + + start.getSecond(); + trivialPart = part2 - part1; + break; + } + case DAY: { + diff = dt.daynr() - start.daynr(); + long part2 = dt.getHour() * 3600 + dt.getMinute() * 60 + dt.getSecond(); + long part1 = start.getHour() * 3600 + start.getMinute() * 60 + start.getSecond(); + trivialPart = part2 - part1; + break; + } + case HOUR: { + diff = (dt.daynr() - start.daynr()) * 24 + (dt.getHour() - start.getHour()); + trivialPart = (dt.getMinute() * 60 + dt.getSecond()) - (start.getMinute() * 60 + start.getSecond()); + break; + } + case MINUTE: { + diff = (dt.daynr() - start.daynr()) * 24 * 60 + (dt.getHour() - start.getHour()) * 60 + + (dt.getMinute() - start.getMinute()); + trivialPart = dt.getSecond() - start.getSecond(); + break; + } + case SECOND: { + diff = (dt.daynr() - start.daynr()) * 24 * 60 * 60 + (dt.getHour() - start.getHour()) * 60 * 60 + + (dt.getMinute() - start.getMinute()) * 60 + (dt.getSecond() - start.getSecond()); + trivialPart = 0; + break; + } + default: + break; + } + + if (isCeil) { + diff = diff + (trivialPart > 0 ? 1 : 0); + } else { + diff = diff - (trivialPart < 0 ? 1 : 0); + } + long deltaInsidePeriod = (diff % periodValue + periodValue) % periodValue; + long step = diff - deltaInsidePeriod; + if (isCeil) { + step = step + (deltaInsidePeriod == 0 ? 0 : periodValue); + } + switch (type) { + case YEAR: + return start.plusYears(step); + case MONTH: + return start.plusMonths(step); + case WEEK: + return start.plusDays(step * 7); + case DAY: + return start.plusDays(step); + case HOUR: + return start.plusHours(step); + case MINUTE: + return start.plusMinutes(step); + case SECOND: + return start.plusSeconds(step); + default: + break; + } + return null; + } + + @FEFunction(name = "week_floor", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral week_floor(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, false, TimeUnit.WEEK); + } + + @FEFunction(name = "week_ceil", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral week_ceil(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, true, TimeUnit.WEEK); + } + + @FEFunction(name = "month_floor", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral month_floor(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, false, TimeUnit.MONTH); + } + + @FEFunction(name = "month_ceil", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral month_ceil(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, true, TimeUnit.MONTH); + } + + @FEFunction(name = "year_floor", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral year_floor(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, false, TimeUnit.YEAR); + } + + @FEFunction(name = "year_ceil", argTypes = { "DATETIMEV2", "INT", "DATETIMEV2" }, returnType = "DATETIMEV2") + public static DateLiteral year_ceil(LiteralExpr datetime, LiteralExpr period, LiteralExpr defaultDatetime) + throws AnalysisException { + return getFloorCeilDateLiteral(datetime, period, defaultDatetime, true, TimeUnit.YEAR); + } + @FEFunction(name = "date_trunc", argTypes = {"DATETIME", "VARCHAR"}, returnType = "DATETIME") public static DateLiteral dateTrunc(LiteralExpr date, LiteralExpr truncate) { if (date.getType().isDateLike()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskDisruptor.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskDisruptor.java index 889355f2cd08c8f..3b59a5187ed3e81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskDisruptor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskDisruptor.java @@ -17,6 +17,7 @@ package org.apache.doris.scheduler.disruptor; +import org.apache.doris.common.Config; import org.apache.doris.scheduler.constants.TaskType; import org.apache.doris.scheduler.manager.TimerJobManager; import org.apache.doris.scheduler.manager.TransientTaskManager; @@ -48,20 +49,15 @@ public class TaskDisruptor implements Closeable { private final Disruptor disruptor; - private static final int DEFAULT_RING_BUFFER_SIZE = 1024; + private static final int DEFAULT_RING_BUFFER_SIZE = Config.async_task_queen_size; + + private static int consumerThreadCount = Config.async_task_consumer_thread_num; /** * The default timeout for {@link #close()} in seconds. */ private static final int DEFAULT_CLOSE_WAIT_TIME_SECONDS = 5; - /** - * The default number of consumers to create for each {@link Disruptor} instance. - */ - private static final int DEFAULT_CONSUMER_COUNT = System.getProperty("event.task.disruptor.consumer.count") - == null ? Runtime.getRuntime().availableProcessors() - : Integer.parseInt(System.getProperty("event.task.disruptor.consumer.count")); - /** * Whether this disruptor has been closed. * if true, then we can't publish any more events. @@ -82,8 +78,8 @@ public TaskDisruptor(TimerJobManager timerJobManager, TransientTaskManager trans ThreadFactory producerThreadFactory = DaemonThreadFactory.INSTANCE; disruptor = new Disruptor<>(TaskEvent.FACTORY, DEFAULT_RING_BUFFER_SIZE, producerThreadFactory, ProducerType.SINGLE, new BlockingWaitStrategy()); - WorkHandler[] workers = new TaskHandler[DEFAULT_CONSUMER_COUNT]; - for (int i = 0; i < DEFAULT_CONSUMER_COUNT; i++) { + WorkHandler[] workers = new TaskHandler[consumerThreadCount]; + for (int i = 0; i < consumerThreadCount; i++) { workers[i] = new TaskHandler(timerJobManager, transientTaskManager); } disruptor.handleEventsWithWorkerPool(workers); diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java index 37ed2143253a41f..297537f712c261b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java @@ -108,11 +108,7 @@ public void onTimerJobTaskHandle(TaskEvent taskEvent) { JobTask jobTask = new JobTask(jobId); try { jobTask.setStartTimeMs(System.currentTimeMillis()); - - - // TODO: We should record the result of the event task. - //Object result = job.getExecutor().execute(); - job.getExecutor().execute(job); + Object result = job.getExecutor().execute(job); job.setLatestCompleteExecuteTimeMs(System.currentTimeMillis()); if (job.isCycleJob()) { updateJobStatusIfPastEndTime(job); @@ -120,9 +116,11 @@ public void onTimerJobTaskHandle(TaskEvent taskEvent) { // one time job should be finished after execute updateOnceTimeJobStatus(job); } + String resultStr = Objects.isNull(result) ? "" : result.toString(); + jobTask.setExecuteResult(resultStr); jobTask.setIsSuccessful(true); } catch (Exception e) { - log.warn("Event job execute failed, jobId: {}, msg : {}", jobId, e.getMessage()); + log.warn("Job execute failed, jobId: {}, msg : {}", jobId, e.getMessage()); job.pause(e.getMessage()); jobTask.setErrorMsg(e.getMessage()); jobTask.setIsSuccessful(false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/SqlJobExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/SqlJobExecutor.java index 2f2495f7f801804..572e895906cee11 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/SqlJobExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/SqlJobExecutor.java @@ -35,6 +35,7 @@ /** * we use this executor to execute sql job + * * @param the state of sql job, we can record the state of sql job */ @Slf4j @@ -50,7 +51,7 @@ public SqlJobExecutor(String sql) { } @Override - public QueryState execute(Job job) throws JobException { + public String execute(Job job) throws JobException { ConnectContext ctx = new ConnectContext(); ctx.setEnv(Env.getCurrentEnv()); ctx.setCluster(ClusterNamespace.getClusterNameFromFullName(job.getDbName())); @@ -66,12 +67,22 @@ public QueryState execute(Job job) throws JobException { try { StmtExecutor executor = new StmtExecutor(ctx, sql); executor.execute(queryId); - log.debug("execute sql job success, sql: {}, state is: {}", sql, ctx.getState()); - return (QueryState) ctx.getState(); + return convertExecuteResult(ctx, taskIdString); } catch (Exception e) { - log.warn("execute sql job failed, sql: {}, error: {}", sql, e); throw new JobException("execute sql job failed, sql: " + sql + ", error: " + e.getMessage()); } } + + private String convertExecuteResult(ConnectContext ctx, String queryId) throws JobException { + if (null == ctx.getState()) { + throw new JobException("execute sql job failed, sql: " + sql + ", error: response state is null"); + } + if (null != ctx.getState().getErrorCode()) { + throw new JobException("error code: " + ctx.getState().getErrorCode() + ", error msg: " + + ctx.getState().getErrorMessage()); + } + return "queryId:" + queryId + ",affectedRows : " + ctx.getState().getAffectedRows() + ", warningRows: " + + ctx.getState().getWarningRows() + ",infoMsg" + ctx.getState().getInfoMessage(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/JobTask.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/JobTask.java index 254875bb7568b8c..3bfe4ffc444125e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/JobTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/JobTask.java @@ -44,6 +44,8 @@ public class JobTask implements Writable { private Long endTimeMs; @SerializedName("successful") private Boolean isSuccessful; + @SerializedName("executeResult") + private String executeResult; @SerializedName("errorMsg") private String errorMsg; @@ -64,6 +66,11 @@ public List getShowInfo() { } else { row.add(isSuccessful ? "SUCCESS" : "FAILED"); } + if (null == executeResult) { + row.add("null"); + } else { + row.add(executeResult); + } if (null == errorMsg) { row.add("null"); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/JobTaskManager.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/JobTaskManager.java index c22c89e6d5a9c13..e18a143ec74ee7f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/JobTaskManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/JobTaskManager.java @@ -40,7 +40,7 @@ @Slf4j public class JobTaskManager implements Writable { - private static final Integer TASK_MAX_NUM = Config.scheduler_job_task_max_num; + private static final Integer TASK_MAX_NUM = Config.scheduler_job_task_max_saved_count; private ConcurrentHashMap> jobTaskMap = new ConcurrentHashMap<>(16); diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TimerJobManager.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TimerJobManager.java index df6736643010616..2b3b922b37b33d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TimerJobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TimerJobManager.java @@ -321,7 +321,13 @@ private List findTasksBetweenTime(Job job, Long endTimeEndWindow, Long nex * We will get the task in the next time window, and then hand it over to the time wheel for timing trigger */ private void executeJobIdsWithinLastTenMinutesWindow() { + // if the task executes for more than 10 minutes, it will be delay, so, + // set lastBatchSchedulerTimestamp to current time + if (lastBatchSchedulerTimestamp + BATCH_SCHEDULER_INTERVAL_MILLI_SECONDS < System.currentTimeMillis()) { + this.lastBatchSchedulerTimestamp = System.currentTimeMillis(); + } if (jobMap.isEmpty()) { + this.lastBatchSchedulerTimestamp += BATCH_SCHEDULER_INTERVAL_MILLI_SECONDS; return; } jobMap.forEach((k, v) -> { diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/ExecuteEnv.java b/fe/fe-core/src/main/java/org/apache/doris/service/ExecuteEnv.java index f7ba622b7e170c5..cf3478e12b3b71a 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/service/ExecuteEnv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/ExecuteEnv.java @@ -31,6 +31,7 @@ public class ExecuteEnv { private MultiLoadMgr multiLoadMgr; private ConnectScheduler scheduler; private long startupTime; + private long processUUID; private List diskInfos; @@ -38,6 +39,7 @@ private ExecuteEnv() { multiLoadMgr = new MultiLoadMgr(); scheduler = new ConnectScheduler(Config.qe_max_connection); startupTime = System.currentTimeMillis(); + processUUID = System.currentTimeMillis(); diskInfos = new ArrayList() {{ add(new FeDiskInfo("meta", Config.meta_dir, DiskUtils.df(Config.meta_dir))); add(new FeDiskInfo("log", Config.sys_log_dir, DiskUtils.df(Config.sys_log_dir))); @@ -69,6 +71,10 @@ public long getStartupTime() { return startupTime; } + public long getProcessUUID() { + return processUUID; + } + public List getDiskInfos() { return this.diskInfos; } 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 27649ef659445e5..6e6e3df5be9b968 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 @@ -21,13 +21,12 @@ import org.apache.doris.analysis.AddColumnsClause; import org.apache.doris.analysis.Analyzer; import org.apache.doris.analysis.ColumnDef; -import org.apache.doris.analysis.InsertStmt; import org.apache.doris.analysis.LabelName; +import org.apache.doris.analysis.NativeInsertStmt; import org.apache.doris.analysis.RestoreStmt; import org.apache.doris.analysis.SetType; import org.apache.doris.analysis.SqlParser; import org.apache.doris.analysis.SqlScanner; -import org.apache.doris.analysis.StatementBase; import org.apache.doris.analysis.TableName; import org.apache.doris.analysis.TypeDef; import org.apache.doris.analysis.UserIdentity; @@ -62,15 +61,12 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.Version; import org.apache.doris.common.annotation.LogException; -import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.SqlParserUtils; import org.apache.doris.common.util.Util; import org.apache.doris.cooldown.CooldownDelete; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.InternalCatalog; -import org.apache.doris.load.EtlJobType; -import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.routineload.RoutineLoadJob; import org.apache.doris.master.MasterImpl; import org.apache.doris.mysql.privilege.AccessControllerManager; @@ -95,7 +91,6 @@ import org.apache.doris.system.Frontend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.tablefunction.MetadataGenerator; -import org.apache.doris.task.LoadEtlTask; import org.apache.doris.task.StreamLoadTask; import org.apache.doris.thrift.FrontendService; import org.apache.doris.thrift.FrontendServiceVersion; @@ -145,6 +140,7 @@ import org.apache.doris.thrift.TInitExternalCtlMetaRequest; import org.apache.doris.thrift.TInitExternalCtlMetaResult; import org.apache.doris.thrift.TListPrivilegesResult; +import org.apache.doris.thrift.TListTableMetadataNameIdsResult; import org.apache.doris.thrift.TListTableStatusResult; import org.apache.doris.thrift.TLoadTxn2PCRequest; import org.apache.doris.thrift.TLoadTxn2PCResult; @@ -184,12 +180,11 @@ import org.apache.doris.thrift.TStreamLoadMultiTablePutResult; import org.apache.doris.thrift.TStreamLoadPutRequest; import org.apache.doris.thrift.TStreamLoadPutResult; -import org.apache.doris.thrift.TStreamLoadWithLoadStatusRequest; -import org.apache.doris.thrift.TStreamLoadWithLoadStatusResult; import org.apache.doris.thrift.TTableIndexQueryStats; +import org.apache.doris.thrift.TTableMetadataNameIds; import org.apache.doris.thrift.TTableQueryStats; import org.apache.doris.thrift.TTableStatus; -import org.apache.doris.thrift.TUniqueId; +import org.apache.doris.thrift.TTxnParams; import org.apache.doris.thrift.TUpdateExportTaskStatusRequest; import org.apache.doris.thrift.TUpdateFollowerStatsCacheRequest; import org.apache.doris.thrift.TWaitingTxnStatusRequest; @@ -199,7 +194,6 @@ import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TransactionState.TxnCoordinator; import org.apache.doris.transaction.TransactionState.TxnSourceType; -import org.apache.doris.transaction.TransactionStatus; import org.apache.doris.transaction.TxnCommitAttachment; import com.google.common.base.Preconditions; @@ -216,13 +210,19 @@ import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.IntSupplier; @@ -332,8 +332,11 @@ public TGetDbsResult getDbNames(TGetDbsParams params) throws TException { LOG.debug("get db request: {}", params); TGetDbsResult result = new TGetDbsResult(); - List dbs = Lists.newArrayList(); - List catalogs = Lists.newArrayList(); + List dbNames = Lists.newArrayList(); + List catalogNames = Lists.newArrayList(); + List dbIds = Lists.newArrayList(); + List catalogIds = Lists.newArrayList(); + PatternMatcher matcher = null; if (params.isSetPattern()) { try { @@ -353,40 +356,51 @@ public TGetDbsResult getDbNames(TGetDbsParams params) throws TException { .getCatalogOrException(params.catalog, catalog -> new TException("Unknown catalog " + catalog))); } for (CatalogIf catalog : catalogIfs) { - List dbNames; + Collection dbs = new HashSet(); try { - dbNames = catalog.getDbNamesOrEmpty(); + dbs = catalog.getAllDbs(); } catch (Exception e) { LOG.warn("failed to get database names for catalog {}", catalog.getName(), e); // Some external catalog may fail to get databases due to wrong connection info. - // So continue here to get databases of other catalogs. + } + LOG.debug("get db size: {}, in catalog: {}", dbs.size(), catalog.getName()); + if (dbs.isEmpty() && params.isSetGetNullCatalog() && params.get_null_catalog) { + catalogNames.add(catalog.getName()); + dbNames.add("NULL"); + catalogIds.add(catalog.getId()); + dbIds.add(-1L); + continue; + } + if (dbs.isEmpty()) { continue; } - LOG.debug("get db names: {}, in catalog: {}", dbNames, catalog.getName()); - UserIdentity currentUser = null; if (params.isSetCurrentUserIdent()) { currentUser = UserIdentity.fromThrift(params.current_user_ident); } else { currentUser = UserIdentity.createAnalyzedUserIdentWithIp(params.user, params.user_ip); } - for (String fullName : dbNames) { + for (DatabaseIf db : dbs) { + String fullName = db.getFullName(); if (!env.getAccessManager().checkDbPriv(currentUser, fullName, PrivPredicate.SHOW)) { continue; } - final String db = ClusterNamespace.getNameFromFullName(fullName); - if (matcher != null && !matcher.match(db)) { + if (matcher != null && !matcher.match(ClusterNamespace.getNameFromFullName(fullName))) { continue; } - catalogs.add(catalog.getName()); - dbs.add(fullName); + catalogNames.add(catalog.getName()); + dbNames.add(fullName); + catalogIds.add(catalog.getId()); + dbIds.add(db.getId()); } } - result.setDbs(dbs); - result.setCatalogs(catalogs); + result.setDbs(dbNames); + result.setCatalogs(catalogNames); + result.setCatalogIds(catalogIds); + result.setDbIds(dbIds); return result; } @@ -687,6 +701,87 @@ public TListTableStatusResult listTableStatus(TGetTablesParams params) throws TE return result; } + public TListTableMetadataNameIdsResult listTableMetadataNameIds(TGetTablesParams params) throws TException { + + LOG.debug("get list simple table request: {}", params); + + TListTableMetadataNameIdsResult result = new TListTableMetadataNameIdsResult(); + List tablesResult = Lists.newArrayList(); + result.setTables(tablesResult); + + UserIdentity currentUser; + if (params.isSetCurrentUserIdent()) { + currentUser = UserIdentity.fromThrift(params.current_user_ident); + } else { + currentUser = UserIdentity.createAnalyzedUserIdentWithIp(params.user, params.user_ip); + } + + String catalogName; + if (params.isSetCatalog()) { + catalogName = params.catalog; + } else { + catalogName = InternalCatalog.INTERNAL_CATALOG_NAME; + } + + PatternMatcher matcher = null; + if (params.isSetPattern()) { + try { + matcher = PatternMatcher.createMysqlPattern(params.getPattern(), + CaseSensibility.TABLE.getCaseSensibility()); + } catch (PatternMatcherException e) { + throw new TException("Pattern is in bad format " + params.getPattern()); + } + } + PatternMatcher finalMatcher = matcher; + + + ExecutorService executor = Executors.newSingleThreadExecutor(); + Future future = executor.submit(() -> { + + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogName); + if (catalog != null) { + DatabaseIf db = catalog.getDbNullable(params.db); + if (db != null) { + List tables = db.getTables(); + for (TableIf table : tables) { + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(currentUser, params.db, + table.getName(), PrivPredicate.SHOW)) { + continue; + } + table.readLock(); + try { + if (finalMatcher != null && !finalMatcher.match(table.getName())) { + continue; + } + TTableMetadataNameIds status = new TTableMetadataNameIds(); + status.setName(table.getName()); + status.setId(table.getId()); + + tablesResult.add(status); + } finally { + table.readUnlock(); + } + } + } + } + }); + try { + if (catalogName.equals(InternalCatalog.INTERNAL_CATALOG_NAME)) { + future.get(); + } else { + future.get(Config.query_metadata_name_ids_timeout, TimeUnit.SECONDS); + } + } catch (TimeoutException e) { + future.cancel(true); + LOG.info("From catalog:{},db:{} get tables timeout.", catalogName, params.db); + } catch (InterruptedException | ExecutionException e) { + future.cancel(true); + } finally { + executor.shutdown(); + } + return result; + } + @Override public TListPrivilegesResult listTablePrivilegeStatus(TGetTablesParams params) throws TException { LOG.debug("get list table privileges request: {}", params); @@ -1703,10 +1798,15 @@ public TStreamLoadPutResult streamLoadPut(TStreamLoadPutRequest request) { TStatus status = new TStatus(TStatusCode.OK); result.setStatus(status); try { - if (Config.enable_pipeline_load) { - result.setPipelineParams(pipelineStreamLoadPutImpl(request)); + if (!Strings.isNullOrEmpty(request.getLoadSql())) { + httpStreamPutImpl(request, result); + return result; } else { - result.setParams(streamLoadPutImpl(request)); + if (Config.enable_pipeline_load) { + result.setPipelineParams(pipelineStreamLoadPutImpl(request)); + } else { + result.setParams(streamLoadPutImpl(request)); + } } } catch (UserException e) { LOG.warn("failed to get stream load plan: {}", e.getMessage()); @@ -1833,25 +1933,31 @@ public TStreamLoadMultiTablePutResult streamLoadMultiTablePut(TStreamLoadPutRequ } - private void streamLoadPutWithSqlImpl(TStreamLoadPutRequest request) throws UserException { - LOG.info("receive stream load put request"); - String loadSql = request.getLoadSql(); - ConnectContext ctx = new ConnectContext(null); + private void httpStreamPutImpl(TStreamLoadPutRequest request, TStreamLoadPutResult result) + throws UserException { + LOG.info("receive http stream put request"); + String originStmt = request.getLoadSql(); + String cluster = request.getCluster(); + if (Strings.isNullOrEmpty(cluster)) { + cluster = SystemInfoService.DEFAULT_CLUSTER; + } + ConnectContext ctx = new ConnectContext(); + if (Strings.isNullOrEmpty(request.getToken())) { + checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(), + request.getUserIp(), PrivPredicate.LOAD); + } ctx.setEnv(Env.getCurrentEnv()); ctx.setQueryId(request.getLoadId()); ctx.setCluster(SystemInfoService.DEFAULT_CLUSTER); ctx.setCurrentUserIdentity(UserIdentity.ROOT); ctx.setQualifiedUser(UserIdentity.ROOT.getQualifiedUser()); - ctx.setThreadLocalInfo(); ctx.setBackendId(request.getBackendId()); - StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request); - ctx.setStreamLoadInfo(streamLoadTask); - ctx.setLoadId(request.getLoadId()); - SqlScanner input = new SqlScanner(new StringReader(loadSql), ctx.getSessionVariable().getSqlMode()); + ctx.setThreadLocalInfo(); + SqlScanner input = new SqlScanner(new StringReader(originStmt), ctx.getSessionVariable().getSqlMode()); SqlParser parser = new SqlParser(input); try { - StatementBase parsedStmt = SqlParserUtils.getFirstStmt(parser); - parsedStmt.setOrigStmt(new OriginStatement(loadSql, 0)); + NativeInsertStmt parsedStmt = (NativeInsertStmt) SqlParserUtils.getFirstStmt(parser); + parsedStmt.setOrigStmt(new OriginStatement(originStmt, 0)); parsedStmt.setUserInfo(ctx.getCurrentUserIdentity()); StmtExecutor executor = new StmtExecutor(ctx, parsedStmt); ctx.setExecutor(executor); @@ -1862,13 +1968,20 @@ private void streamLoadPutWithSqlImpl(TStreamLoadPutRequest request) throws User coord.setLoadMemLimit(request.getExecMemLimit()); coord.setQueryType(TQueryType.LOAD); QeProcessorImpl.INSTANCE.registerQuery(request.getLoadId(), coord); - coord.exec(); + + TExecPlanFragmentParams plan = coord.getStreamLoadPlan(); + final long txn_id = parsedStmt.getTransactionId(); + result.setParams(plan); + result.getParams().setDbName(parsedStmt.getDbName()); + result.getParams().setTableName(parsedStmt.getTbl()); + // The txn_id here is obtained from the NativeInsertStmt + result.getParams().setTxnConf(new TTxnParams().setTxnId(txn_id)); } catch (UserException e) { - LOG.warn("exec sql error {}", e.getMessage()); - throw new UserException("exec sql error"); + LOG.warn("exec sql error", e); + throw new UserException("exec sql error" + e); } catch (Throwable e) { LOG.warn("exec sql error catch unknown result.", e); - throw new UserException("exec sql error catch unknown result"); + throw new UserException("exec sql error catch unknown result." + e); } } @@ -1923,6 +2036,7 @@ private TExecPlanFragmentParams generatePlanFragmentParams(TStreamLoadPutRequest } txnState.addTableIndexes(table); plan.setTableName(table.getName()); + plan.query_options.setFeProcessUuid(ExecuteEnv.getInstance().getProcessUUID()); return plan; } finally { table.readUnlock(); @@ -1988,93 +2102,6 @@ private TPipelineFragmentParams generatePipelineStreamLoadPut(TStreamLoadPutRequ } } - // this function need to be improved - @Override - public TStreamLoadWithLoadStatusResult streamLoadWithLoadStatus(TStreamLoadWithLoadStatusRequest request) { - TStreamLoadWithLoadStatusResult result = new TStreamLoadWithLoadStatusResult(); - TUniqueId loadId = request.getLoadId(); - Coordinator coord = QeProcessorImpl.INSTANCE.getCoordinator(loadId); - long totalRows = 0; - long loadedRows = 0; - int filteredRows = 0; - int unselectedRows = 0; - long txnId = -1; - Throwable throwable = null; - String label = ""; - if (coord == null) { - result.setStatus(new TStatus(TStatusCode.RUNTIME_ERROR)); - LOG.info("runtime error, query {} does not exist", DebugUtil.printId(loadId)); - return result; - } - ConnectContext context = coord.getConnectContext(); - StmtExecutor exec = context.getExecutor(); - InsertStmt insertStmt = (InsertStmt) exec.getParsedStmt(); - label = insertStmt.getLabel(); - txnId = insertStmt.getTransactionId(); - result.setTxnId(txnId); - TransactionStatus txnStatus = TransactionStatus.ABORTED; - if (coord.getExecStatus().ok()) { - if (coord.getLoadCounters().get(LoadEtlTask.DPP_NORMAL_ALL) != null) { - totalRows = Long.parseLong(coord.getLoadCounters().get(LoadEtlTask.DPP_NORMAL_ALL)); - } - if (coord.getLoadCounters().get(LoadEtlTask.DPP_ABNORMAL_ALL) != null) { - filteredRows = Integer.parseInt(coord.getLoadCounters().get(LoadEtlTask.DPP_ABNORMAL_ALL)); - } - if (coord.getLoadCounters().get(LoadJob.UNSELECTED_ROWS) != null) { - unselectedRows = Integer.parseInt(coord.getLoadCounters().get(LoadJob.UNSELECTED_ROWS)); - } - loadedRows = totalRows - filteredRows - unselectedRows; - try { - if (Env.getCurrentGlobalTransactionMgr().commitAndPublishTransaction( - insertStmt.getDbObj(), Lists.newArrayList(insertStmt.getTargetTable()), - insertStmt.getTransactionId(), - TabletCommitInfo.fromThrift(coord.getCommitInfos()), - context.getSessionVariable().getInsertVisibleTimeoutMs())) { - txnStatus = TransactionStatus.VISIBLE; - } else { - txnStatus = TransactionStatus.COMMITTED; - } - } catch (Throwable t) { - // if any throwable being thrown during insert operation, first we should abort this txn - LOG.warn("handle insert stmt fail: {}", label, t); - try { - Env.getCurrentGlobalTransactionMgr().abortTransaction( - insertStmt.getDbObj().getId(), insertStmt.getTransactionId(), - t.getMessage() == null ? "unknown reason" : t.getMessage()); - } catch (Exception abortTxnException) { - // just print a log if abort txn failed. This failure do not need to pass to user. - // user only concern abort how txn failed. - LOG.warn("errors when abort txn", abortTxnException); - } - throwable = t; - } finally { - QeProcessorImpl.INSTANCE.unregisterQuery(loadId); - } - try { - context.getEnv().getLoadManager() - .recordFinishedLoadJob(label, txnId, insertStmt.getDbName(), - insertStmt.getTargetTable().getId(), - EtlJobType.INSERT, System.currentTimeMillis(), - throwable == null ? "" : throwable.getMessage(), - coord.getTrackingUrl(), insertStmt.getUserInfo()); - } catch (MetaNotFoundException e) { - LOG.warn("Record info of insert load with error {}", e.getMessage(), e); - } - context.setOrUpdateInsertResult(txnId, label, insertStmt.getDbName(), insertStmt.getTbl(), - txnStatus, loadedRows, filteredRows); - context.updateReturnRows((int) loadedRows); - result.setStatus(new TStatus(TStatusCode.OK)); - result.setTotalRows(totalRows); - result.setLoadedRows(loadedRows); - result.setFilteredRows(filteredRows); - result.setUnselectedRows(unselectedRows); - } else { - QeProcessorImpl.INSTANCE.unregisterQuery(loadId); - result.setStatus(new TStatus(TStatusCode.CANCELLED)); - } - return result; - } - @Override public TStatus snapshotLoaderReport(TSnapshotLoaderReportRequest request) throws TException { if (Env.getCurrentEnv().getBackupHandler().report(request.getTaskType(), request.getJobId(), @@ -2111,6 +2138,7 @@ public TFrontendPingFrontendResult ping(TFrontendPingFrontendRequest request) th result.setRpcPort(Config.rpc_port); result.setVersion(Version.DORIS_BUILD_VERSION + "-" + Version.DORIS_BUILD_SHORT_HASH); result.setLastStartupTime(exeEnv.getStartupTime()); + result.setProcessUUID(exeEnv.getProcessUUID()); if (exeEnv.getDiskInfos() != null) { result.setDiskInfos(FeDiskInfo.toThrifts(exeEnv.getDiskInfos())); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java index 73441db2e2f8f7d..8f3348064018b11 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfo.java @@ -200,6 +200,9 @@ public AnalysisInfo(long jobId, long taskId, List taskIds, String catalogN this.partitionOnly = partitionOnly; this.samplingPartition = samplingPartition; this.cronExpression = cronExpression; + if (cronExpression != null) { + this.cronExprStr = cronExpression.getCronExpression(); + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java index dc368fcdf1fb59a..081ee4554cb4c24 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisInfoBuilder.java @@ -89,6 +89,7 @@ public AnalysisInfoBuilder(AnalysisInfo info) { externalTableLevelTask = info.externalTableLevelTask; partitionOnly = info.partitionOnly; samplingPartition = info.samplingPartition; + cronExpression = info.cronExpression; } public AnalysisInfoBuilder setJobId(long jobId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index 5fbfb56829548d2..bdd325e6d1aeba1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -347,7 +347,9 @@ public AnalysisInfo buildAndAssignJob(AnalyzeTblStmt stmt) throws DdlException { analysisJobIdToTaskMap.put(jobInfo.jobId, analysisTaskInfos); // TODO: maybe we should update table stats only when all task succeeded. updateTableStats(jobInfo); - analysisTaskInfos.values().forEach(taskExecutor::submitTask); + if (!jobInfo.scheduleType.equals(ScheduleType.PERIOD)) { + analysisTaskInfos.values().forEach(taskExecutor::submitTask); + } return jobInfo; } @@ -464,7 +466,7 @@ private Map> validateAndGetPartitions(TableIf table, Set 0 ? numBuckets : StatisticConstants.HISTOGRAM_MAX_BUCKET_NUM; - info.setMaxBucketNum(maxBucketNum); + infoBuilder.setMaxBucketNum(maxBucketNum); } long periodTimeInMs = stmt.getPeriodTimeInMs(); - info.setPeriodTimeInMs(periodTimeInMs); + infoBuilder.setPeriodTimeInMs(periodTimeInMs); Map> colToPartitions = validateAndGetPartitions(table, columnNames, partitionNames, analysisType, analysisMode); - info.setColToPartitions(colToPartitions); - info.setTaskIds(Lists.newArrayList()); + infoBuilder.setColToPartitions(colToPartitions); + infoBuilder.setTaskIds(Lists.newArrayList()); - return info.build(); + return infoBuilder.build(); } @VisibleForTesting @@ -820,7 +822,7 @@ public void execute(ThreadPoolExecutor executor) { public List findPeriodicJobs() { synchronized (analysisJobInfoMap) { Predicate p = a -> { - if (a.state.equals(AnalysisState.RUNNING) || a.state.equals(AnalysisState.PENDING)) { + if (a.state.equals(AnalysisState.RUNNING)) { return false; } if (a.cronExpression == null) { @@ -844,7 +846,8 @@ public List findTasks(long jobId) { public List findTasksByTaskIds(long jobId) { AnalysisInfo jobInfo = analysisJobInfoMap.get(jobId); if (jobInfo != null && jobInfo.taskIds != null) { - return jobInfo.taskIds.stream().map(id -> analysisTaskInfoMap.get(id)).collect(Collectors.toList()); + return jobInfo.taskIds.stream().map(analysisTaskInfoMap::get).filter(i -> i != null) + .collect(Collectors.toList()); } return null; } @@ -944,6 +947,7 @@ public void logCreateTableStats(TableStats tableStats) { } public void registerSysJob(AnalysisInfo jobInfo, Map taskInfos) { + jobInfo.state = AnalysisState.RUNNING; systemJobInfoMap.put(jobInfo.jobId, jobInfo); analysisJobIdToTaskMap.put(jobInfo.jobId, taskInfos); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisState.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisState.java index bab8a462e8a5bd7..3abc4c224faad2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisState.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisState.java @@ -18,7 +18,9 @@ package org.apache.doris.statistics; public enum AnalysisState { + // When analyze job/task created, but never run PENDING, + // When analyze job/task is in running queue RUNNING, FINISHED, FAILED; diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java index 235861119ab92d9..032392908177da7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/BaseAnalysisTask.java @@ -236,6 +236,7 @@ protected void executeWithExceptionOnFail(StmtExecutor stmtExecutor) throws Exce if (killed) { return; } + LOG.debug("execute internal sql: {}", stmtExecutor.getOriginStmt()); stmtExecutor.execute(); QueryState queryState = stmtExecutor.getContext().getState(); if (queryState.getStateType().equals(MysqlStateType.ERR)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java index 85965e9513d0ac9..80d33e7c85f7fb4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.Type; +import org.apache.doris.common.AnalysisException; import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.base.Preconditions; @@ -168,21 +169,31 @@ public static ColumnStatistic fromResultRow(ResultRow row) { String min = row.get(10); String max = row.get(11); if (min != null && !min.equalsIgnoreCase("NULL")) { - columnStatisticBuilder.setMinValue(StatisticsUtil.convertToDouble(col.getType(), min)); - columnStatisticBuilder.setMinExpr(StatisticsUtil.readableValue(col.getType(), min)); + try { + columnStatisticBuilder.setMinValue(StatisticsUtil.convertToDouble(col.getType(), min)); + columnStatisticBuilder.setMinExpr(StatisticsUtil.readableValue(col.getType(), min)); + } catch (AnalysisException e) { + LOG.warn("Failed to deserialize column {} min value {}.", col, min, e); + columnStatisticBuilder.setMinValue(Double.MIN_VALUE); + } } else { columnStatisticBuilder.setMinValue(Double.MIN_VALUE); } if (max != null && !max.equalsIgnoreCase("NULL")) { - columnStatisticBuilder.setMaxValue(StatisticsUtil.convertToDouble(col.getType(), max)); - columnStatisticBuilder.setMaxExpr(StatisticsUtil.readableValue(col.getType(), max)); + try { + columnStatisticBuilder.setMaxValue(StatisticsUtil.convertToDouble(col.getType(), max)); + columnStatisticBuilder.setMaxExpr(StatisticsUtil.readableValue(col.getType(), max)); + } catch (AnalysisException e) { + LOG.warn("Failed to deserialize column {} max value {}.", col, max, e); + columnStatisticBuilder.setMaxValue(Double.MAX_VALUE); + } } else { columnStatisticBuilder.setMaxValue(Double.MAX_VALUE); } columnStatisticBuilder.setUpdatedTime(row.get(13)); return columnStatisticBuilder.build(); } catch (Exception e) { - LOG.warn("Failed to deserialize column statistics, column not exists", e); + LOG.warn("Failed to deserialize column statistics.", e); return ColumnStatistic.UNKNOWN; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java index 257a104254d081f..ef26d7349e5bd01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/OlapAnalysisTask.java @@ -30,6 +30,7 @@ import org.apache.commons.text.StringSubstitutor; import java.util.ArrayList; +import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -109,6 +110,8 @@ public void doExecute() throws Exception { @VisibleForTesting public void execSQLs(List partitionAnalysisSQLs, Map params) throws Exception { + long startTime = System.currentTimeMillis(); + LOG.debug("analyze task {} start at {}", info.toString(), new Date()); try (AutoCloseConnectContext r = StatisticsUtil.buildConnectContext()) { List> sqlGroups = Lists.partition(partitionAnalysisSQLs, StatisticConstants.UNION_ALL_LIMIT); for (List group : sqlGroups) { @@ -127,80 +130,25 @@ public void execSQLs(List partitionAnalysisSQLs, Map par queryState.getErrorMessage())); } } - if (buf.size() > 1) { - for (List colStatsDataList : buf) { - StringBuilder batchInsertSQL = - new StringBuilder("INSERT INTO __internal_schema.column_statistics VALUES "); - StringJoiner sj = new StringJoiner(","); - colStatsDataList.forEach(c -> sj.add(c.toSQL(true))); - batchInsertSQL.append(sj.toString()); - stmtExecutor = new StmtExecutor(r.connectContext, batchInsertSQL.toString()); - executeWithExceptionOnFail(stmtExecutor); - } - params.put("type", col.getType().toString()); - StringSubstitutor stringSubstitutor = new StringSubstitutor(params); - String sql = stringSubstitutor.replace(ANALYZE_COLUMN_SQL_TEMPLATE); - stmtExecutor = new StmtExecutor(r.connectContext, sql); - executeWithExceptionOnFail(stmtExecutor); - } else { - List colStatsDataList = buf.get(0); - String batchInsertSQLTemplate = "INSERT INTO __internal_schema.column_statistics " - + "SELECT id, catalog_id, db_id, tbl_id, idx_id, col_id, part_id, row_count," - + "ndv, null_count, CAST(min AS string), CAST(max AS string), data_size, update_time FROM (" - + "SELECT CONCAT(${tblId}, '-', ${idxId}, '-', '${colId}') AS id, " - + " ${catalogId} AS catalog_id, " - + " ${dbId} AS db_id, " - + " ${tblId} AS tbl_id, " - + " ${idxId} AS idx_id, " - + " '${colId}' AS col_id, " - + " NULL AS part_id, " - + " SUM(count) AS row_count," - + " SUM(null_count) AS null_count, " - + " MIN(CAST (min AS ${type})) AS min, " - + " MAX(CAST (max AS ${type})) AS max, " - + " SUM(data_size_in_bytes) AS data_size, " - + " NOW() AS update_time" - + " FROM (${partitionStatsView}) psv) t1, " - + " (SELECT NDV(`${colName}`) AS ndv " - + " FROM `${dbName}`.`${tblName}` ${sampleExpr}) t2 UNION ALL ${partitionStatsView}"; - StringJoiner sj = new StringJoiner(" UNION ALL "); - String selectPartitionTemplate = - "SELECT %s AS id," - + "%s AS catalog_id," - + "%s AS db_id," - + "%s AS tbl_id," - + "%s AS idx_id," - + "%s AS col_id," - + "%s AS part_id," - + "%s AS count," - + "%s AS ndv," - + "%s AS null_count," - + "%s as min," - + "%s as max," - + "%s as data_size_in_bytes," - + "%s AS update_time"; - colStatsDataList.forEach(c -> sj.add(String.format(selectPartitionTemplate, - StatisticsUtil.quote(c.statsId.id), - c.statsId.catalogId, - c.statsId.dbId, - c.statsId.tblId, - c.statsId.idxId, - StatisticsUtil.quote(c.statsId.colId), - c.statsId.partId, - c.count, - c.ndv, - c.nullCount, - c.minLit == null ? null : StatisticsUtil.quote(StatisticsUtil.escapeSQL(c.minLit)), - c.maxLit == null ? null : StatisticsUtil.quote(StatisticsUtil.escapeSQL(c.maxLit)), - c.dataSizeInBytes, - StatisticsUtil.quote(c.updateTime)))); - params.put("partitionStatsView", sj.toString()); - params.put("type", col.getType().toString()); - StringSubstitutor stringSubstitutor = new StringSubstitutor(params); - String insertSQL = stringSubstitutor.replace(batchInsertSQLTemplate); - stmtExecutor = new StmtExecutor(r.connectContext, insertSQL); + for (List colStatsDataList : buf) { + StringBuilder batchInsertSQL = + new StringBuilder("INSERT INTO " + StatisticConstants.FULL_QUALIFIED_STATS_TBL_NAME + + " VALUES "); + StringJoiner sj = new StringJoiner(","); + colStatsDataList.forEach(c -> sj.add(c.toSQL(true))); + batchInsertSQL.append(sj.toString()); + stmtExecutor = new StmtExecutor(r.connectContext, batchInsertSQL.toString()); executeWithExceptionOnFail(stmtExecutor); } + params.put("type", col.getType().toString()); + StringSubstitutor stringSubstitutor = new StringSubstitutor(params); + String sql = stringSubstitutor.replace(ANALYZE_COLUMN_SQL_TEMPLATE); + stmtExecutor = new StmtExecutor(r.connectContext, sql); + executeWithExceptionOnFail(stmtExecutor); + } finally { + LOG.debug("analyze task {} end. cost {}ms", info, + System.currentTimeMillis() - startTime); } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java index ce512c1952bf400..f0f6f60277783a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java @@ -28,15 +28,12 @@ public class StatisticConstants { public static final String STATISTIC_TBL_NAME = "column_statistics"; - public static final String HISTOGRAM_TBL_NAME = "histogram_statistics"; public static final int MAX_NAME_LEN = 64; public static final int ID_LEN = 4096; - public static final int STATISTICS_CACHE_VALID_DURATION_IN_HOURS = 24 * 2; - public static final int STATISTICS_CACHE_REFRESH_INTERVAL = 24 * 2; public static final int ROW_COUNT_CACHE_VALID_DURATION_IN_HOURS = 12; @@ -76,6 +73,8 @@ public class StatisticConstants { public static final String DB_NAME = SystemInfoService.DEFAULT_CLUSTER + ":" + FeConstants.INTERNAL_DB_NAME; + public static final String FULL_QUALIFIED_STATS_TBL_NAME = FeConstants.INTERNAL_DB_NAME + "." + STATISTIC_TBL_NAME; + public static final int STATISTIC_INTERNAL_TABLE_REPLICA_NUM = 3; public static final int RETRY_LOAD_QUEUE_SIZE = 1000; diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoAnalyzer.java index e07a4c1ddf5610e..5d704e4f3b9f17d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoAnalyzer.java @@ -21,7 +21,6 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.View; import org.apache.doris.common.Config; @@ -53,7 +52,7 @@ public class StatisticsAutoAnalyzer extends MasterDaemon { private static final Logger LOG = LogManager.getLogger(StatisticsAutoAnalyzer.class); - private AnalysisTaskExecutor analysisTaskExecutor; + private final AnalysisTaskExecutor analysisTaskExecutor; public StatisticsAutoAnalyzer() { super("Automatic Analyzer", @@ -70,6 +69,7 @@ protected void runAfterCatalogReady() { if (!StatisticsUtil.statsTblAvailable()) { return; } + analyzePeriodically(); if (!checkAnalyzeTime(LocalTime.now(TimeUtils.getTimeZone().toZoneId()))) { return; } @@ -78,7 +78,6 @@ protected void runAfterCatalogReady() { return; } - analyzePeriodically(); if (Config.enable_full_auto_analyze) { analyzeAll(); } @@ -88,7 +87,9 @@ protected void runAfterCatalogReady() { private void analyzeAll() { Set catalogs = Env.getCurrentEnv().getCatalogMgr().getCopyOfCatalog(); for (CatalogIf ctl : catalogs) { - + if (!ctl.enableAutoAnalyze()) { + continue; + } Collection dbs = ctl.getAllDbs(); for (DatabaseIf databaseIf : dbs) { if (StatisticConstants.STATISTICS_DB_BLACK_LIST.contains(databaseIf.getFullName())) { @@ -145,10 +146,9 @@ private void analyzePeriodically() { AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); List jobInfos = analysisManager.findPeriodicJobs(); for (AnalysisInfo jobInfo : jobInfos) { - jobInfo = new AnalysisInfoBuilder(jobInfo).setJobType(JobType.SYSTEM).build(); createSystemAnalysisJob(jobInfo); } - } catch (DdlException e) { + } catch (Exception e) { LOG.warn("Failed to periodically analyze the statistics." + e); } } @@ -159,11 +159,11 @@ protected AnalysisInfo getReAnalyzeRequiredPart(AnalysisInfo jobInfo) { .findTable(jobInfo.catalogName, jobInfo.dbName, jobInfo.tblName); TableStats tblStats = Env.getCurrentEnv().getAnalysisManager().findTableStatsStatus(table.getId()); - if (!(tblStats == null || needReanalyzeTable(table, tblStats))) { + if (!(tblStats == null || table.needReAnalyzeTable(tblStats))) { return null; } - Set needRunPartitions = findReAnalyzeNeededPartitions(table, tblStats); + Set needRunPartitions = table.findReAnalyzeNeededPartitions(tblStats); if (needRunPartitions.isEmpty()) { return null; @@ -172,31 +172,6 @@ protected AnalysisInfo getReAnalyzeRequiredPart(AnalysisInfo jobInfo) { return getAnalysisJobInfo(jobInfo, table, needRunPartitions); } - @VisibleForTesting - protected Set findReAnalyzeNeededPartitions(TableIf table, TableStats tableStats) { - if (tableStats == null) { - return table.getPartitionNames().stream().map(table::getPartition) - .filter(Partition::hasData).map(Partition::getName).collect(Collectors.toSet()); - } - return table.getPartitionNames().stream() - .map(table::getPartition) - .filter(Partition::hasData) - .filter(partition -> - partition.getVisibleVersionTime() >= tableStats.updatedTime).map(Partition::getName) - .collect(Collectors.toSet()); - } - - private boolean needReanalyzeTable(TableIf table, TableStats tblStats) { - long rowCount = table.getRowCount(); - // TODO: Do we need to analyze an empty table? - if (rowCount == 0) { - return false; - } - long updateRows = tblStats.updatedRows.get(); - int tblHealth = StatisticsUtil.getTableHealth(rowCount, updateRows); - return tblHealth < Config.table_stats_health_threshold; - } - @VisibleForTesting public AnalysisInfo getAnalysisJobInfo(AnalysisInfo jobInfo, TableIf table, Set needRunPartitions) { @@ -238,7 +213,7 @@ private boolean checkAnalyzeTime(LocalTime now) { // Analysis job created by the system @VisibleForTesting - public void createSystemAnalysisJob(AnalysisInfo jobInfo) + protected void createSystemAnalysisJob(AnalysisInfo jobInfo) throws DdlException { if (jobInfo.colToPartitions.isEmpty()) { // No statistics need to be collected or updated @@ -248,6 +223,9 @@ public void createSystemAnalysisJob(AnalysisInfo jobInfo) Map analysisTaskInfos = new HashMap<>(); AnalysisManager analysisManager = Env.getCurrentEnv().getAnalysisManager(); analysisManager.createTaskForEachColumns(jobInfo, analysisTaskInfos, false); + if (StatisticsUtil.isExternalTable(jobInfo.catalogName, jobInfo.dbName, jobInfo.tblName)) { + analysisManager.createTableLevelTaskForExternalTable(jobInfo, analysisTaskInfos, false); + } Env.getCurrentEnv().getAnalysisManager().registerSysJob(jobInfo, analysisTaskInfos); analysisTaskInfos.values().forEach(analysisTaskExecutor::submitTask); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java index 8d09475bcb08d5a..87d8a0ba15c9c92 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java @@ -170,6 +170,7 @@ public static AutoCloseConnectContext buildConnectContext() { sessionVariable.enableProfile = false; sessionVariable.queryTimeoutS = Config.analyze_task_timeout_in_hours * 60 * 60; sessionVariable.insertTimeoutS = Config.analyze_task_timeout_in_hours * 60 * 60; + sessionVariable.enableFileCache = false; connectContext.setEnv(Env.getCurrentEnv()); connectContext.setDatabase(FeConstants.INTERNAL_DB_NAME); connectContext.setQualifiedUser(UserIdentity.ROOT.getQualifiedUser()); @@ -488,9 +489,10 @@ public static int getTableHealth(long totalRows, long updatedRows) { * First get it from remote table parameters. If not found, estimate it : totalSize/estimatedRowSize * * @param table Hive HMSExternalTable to estimate row count. + * @param isInit Flag to indicate if this is called during init. To avoid recursively get schema. * @return estimated row count */ - public static long getHiveRowCount(HMSExternalTable table) { + public static long getHiveRowCount(HMSExternalTable table, boolean isInit) { Map parameters = table.getRemoteTable().getParameters(); if (parameters == null) { return -1; @@ -499,7 +501,7 @@ public static long getHiveRowCount(HMSExternalTable table) { if (parameters.containsKey(NUM_ROWS)) { return Long.parseLong(parameters.get(NUM_ROWS)); } - if (!parameters.containsKey(TOTAL_SIZE)) { + if (!parameters.containsKey(TOTAL_SIZE) || isInit) { return -1; } // Table parameters doesn't contain row count but contain total size. Estimate row count : totalSize/rowSize diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/Frontend.java b/fe/fe-core/src/main/java/org/apache/doris/system/Frontend.java index 35964f2b2a6906d..95937b9d42013a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/Frontend.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/Frontend.java @@ -60,6 +60,8 @@ public class Frontend implements Writable { private boolean isAlive = false; + private long processUUID = 0; + public Frontend() { } @@ -122,6 +124,10 @@ public long getLastStartupTime() { return lastStartupTime; } + public long getProcessUUID() { + return processUUID; + } + public long getLastUpdateTime() { return lastUpdateTime; } @@ -150,10 +156,16 @@ public boolean handleHbResponse(FrontendHbResponse hbResponse, boolean isReplay) replayedJournalId = hbResponse.getReplayedJournalId(); lastUpdateTime = hbResponse.getHbTime(); heartbeatErrMsg = ""; - lastStartupTime = hbResponse.getFeStartTime(); + lastStartupTime = hbResponse.getProcessUUID(); diskInfos = hbResponse.getDiskInfos(); isChanged = true; + processUUID = hbResponse.getProcessUUID(); } else { + // A non-master node disconnected. + // Set startUUID to zero, and be's heartbeat mgr will ignore this hb, + // so that its cancel worker will not cancel queries from this fe immediately + // until it receives a valid start UUID. + processUUID = 0; if (isAlive) { isAlive = false; isChanged = true; diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/FrontendHbResponse.java b/fe/fe-core/src/main/java/org/apache/doris/system/FrontendHbResponse.java index 8ad4f36e6dde3b9..f7d7e90624d0228 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/FrontendHbResponse.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/FrontendHbResponse.java @@ -19,6 +19,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import org.apache.doris.service.ExecuteEnv; import org.apache.doris.service.FeDiskInfo; import com.google.gson.annotations.SerializedName; @@ -42,6 +43,7 @@ public class FrontendHbResponse extends HeartbeatResponse implements Writable { private long replayedJournalId; private String version; private long feStartTime; + private long processUUID; private List diskInfos; public FrontendHbResponse() { @@ -50,7 +52,8 @@ public FrontendHbResponse() { public FrontendHbResponse(String name, int queryPort, int rpcPort, long replayedJournalId, long hbTime, String version, - long feStartTime, List diskInfos) { + long feStartTime, List diskInfos, + long processUUID) { super(HeartbeatResponse.Type.FRONTEND); this.status = HbStatus.OK; this.name = name; @@ -59,8 +62,9 @@ public FrontendHbResponse(String name, int queryPort, int rpcPort, this.replayedJournalId = replayedJournalId; this.hbTime = hbTime; this.version = version; - this.feStartTime = feStartTime; + this.processUUID = processUUID; this.diskInfos = diskInfos; + this.processUUID = processUUID; } public FrontendHbResponse(String name, String errMsg) { @@ -68,6 +72,7 @@ public FrontendHbResponse(String name, String errMsg) { this.status = HbStatus.BAD; this.name = name; this.msg = errMsg; + this.processUUID = ExecuteEnv.getInstance().getProcessUUID(); } public String getName() { @@ -90,6 +95,10 @@ public String getVersion() { return version; } + public long getProcessUUID() { + return processUUID; + } + public long getFeStartTime() { return feStartTime; } @@ -116,7 +125,7 @@ public String toString() { sb.append(", queryPort: ").append(queryPort); sb.append(", rpcPort: ").append(rpcPort); sb.append(", replayedJournalId: ").append(replayedJournalId); - sb.append(", festartTime: ").append(feStartTime); + sb.append(", festartTime: ").append(processUUID); return sb.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java index 6cfcff34e093bb7..cc2730328a22c6c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java @@ -39,6 +39,7 @@ import org.apache.doris.thrift.TBrokerOperationStatusCode; import org.apache.doris.thrift.TBrokerPingBrokerRequest; import org.apache.doris.thrift.TBrokerVersion; +import org.apache.doris.thrift.TFrontendInfo; import org.apache.doris.thrift.TFrontendPingFrontendRequest; import org.apache.doris.thrift.TFrontendPingFrontendResult; import org.apache.doris.thrift.TFrontendPingFrontendStatusCode; @@ -101,11 +102,12 @@ public void setMaster(int clusterId, String token, long epoch) { */ @Override protected void runAfterCatalogReady() { + // Get feInfos of previous iteration. + List feInfos = Env.getCurrentEnv().getFrontendInfos(); List> hbResponses = Lists.newArrayList(); - // send backend heartbeat for (Backend backend : nodeMgr.getIdToBackend().values()) { - BackendHeartbeatHandler handler = new BackendHeartbeatHandler(backend); + BackendHeartbeatHandler handler = new BackendHeartbeatHandler(backend, feInfos); hbResponses.add(executor.submit(handler)); } @@ -204,9 +206,11 @@ private boolean handleHbResponse(HeartbeatResponse response, boolean isReplay) { // backend heartbeat private class BackendHeartbeatHandler implements Callable { private Backend backend; + private List feInfos; - public BackendHeartbeatHandler(Backend backend) { + public BackendHeartbeatHandler(Backend backend, List feInfos) { this.backend = backend; + this.feInfos = feInfos; } @Override @@ -222,6 +226,7 @@ public HeartbeatResponse call() { long flags = heartbeatFlags.getHeartbeatFlags(); copiedMasterInfo.setHeartbeatFlags(flags); copiedMasterInfo.setBackendId(backendId); + copiedMasterInfo.setFrontendInfos(feInfos); THeartbeatResult result; if (!FeConstants.runningUnitTest) { client = ClientPool.backendHeartbeatPool.borrowObject(beAddr); @@ -301,7 +306,8 @@ public HeartbeatResponse call() { return new FrontendHbResponse(fe.getNodeName(), Config.query_port, Config.rpc_port, Env.getCurrentEnv().getMaxJournalId(), System.currentTimeMillis(), Version.DORIS_BUILD_VERSION + "-" + Version.DORIS_BUILD_SHORT_HASH, - ExecuteEnv.getInstance().getStartupTime(), ExecuteEnv.getInstance().getDiskInfos()); + ExecuteEnv.getInstance().getStartupTime(), ExecuteEnv.getInstance().getDiskInfos(), + ExecuteEnv.getInstance().getProcessUUID()); } else { return new FrontendHbResponse(fe.getNodeName(), "not ready"); } @@ -323,7 +329,7 @@ private HeartbeatResponse getHeartbeatResponse() { return new FrontendHbResponse(fe.getNodeName(), result.getQueryPort(), result.getRpcPort(), result.getReplayedJournalId(), System.currentTimeMillis(), result.getVersion(), result.getLastStartupTime(), - FeDiskInfo.fromThrifts(result.getDiskInfos())); + FeDiskInfo.fromThrifts(result.getDiskInfos()), result.getProcessUUID()); } else { return new FrontendHbResponse(fe.getNodeName(), result.getMsg()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java index 8826fbbf64ab389..5cc6b36f87d0826 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java @@ -105,7 +105,7 @@ public void setPort(int port) { } public String getIdent() { - return host; + return host + "_" + port; } @Override 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 1511667719bbaea..d1fd486679bc778 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 @@ -226,6 +226,15 @@ protected void parseProperties(Map validParams) throws AnalysisE throw new AnalysisException("format:" + formatString + " is not supported."); } + // TODO Support is needed in the future + if (getTFileType() == TFileType.FILE_STREAM && (formatString.equals("json") + || formatString.equals("csv_with_names") + || formatString.equals("csv_with_names_and_types") + || formatString.equals("parquet") + || formatString.equals("avro") + || formatString.equals("orc"))) { + throw new AnalysisException("current http_stream does not yet support json, parquet and orc"); + } columnSeparator = validParams.getOrDefault(COLUMN_SEPARATOR, DEFAULT_COLUMN_SEPARATOR); lineDelimiter = validParams.getOrDefault(LINE_DELIMITER, DEFAULT_LINE_DELIMITER); jsonRoot = validParams.getOrDefault(JSON_ROOT, ""); @@ -405,6 +414,15 @@ public List getTableColumns() throws AnalysisException { } protected Backend getBackend() { + ConnectContext ctx = ConnectContext.get(); + // For the http stream task, we should obtain the be for processing the task + long backendId = ctx.getBackendId(); + if (getTFileType() == TFileType.FILE_STREAM) { + Backend be = Env.getCurrentSystemInfo().getIdToBackend().get(backendId); + if (be.isAlive()) { + return be; + } + } for (Backend be : Env.getCurrentSystemInfo().getIdToBackend().values()) { if (be.isAlive()) { return be; @@ -477,7 +495,13 @@ private PFetchTableSchemaRequest getFetchTableStructureRequest() throws Analysis fileScanRangeParams.setProperties(locationProperties); fileScanRangeParams.setFileAttributes(getFileAttributes()); ConnectContext ctx = ConnectContext.get(); - fileScanRangeParams.setLoadId(ctx.getLoadId()); + fileScanRangeParams.setLoadId(ctx.queryId()); + + if (getTFileType() == TFileType.FILE_STREAM) { + fileStatuses.add(new TBrokerFileStatus("", false, -1, true)); + fileScanRangeParams.setFileType(getTFileType()); + } + if (getTFileType() == TFileType.FILE_HDFS) { THdfsParams tHdfsParams = HdfsResource.generateHdfsParam(locationProperties); String fsNmae = getLocationProperties().get(HdfsResource.HADOOP_FS_NAME); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/StreamTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java similarity index 76% rename from fe/fe-core/src/main/java/org/apache/doris/tablefunction/StreamTableValuedFunction.java rename to fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java index 0acd6284bed9ff8..8a4c5cb2d81adc6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/StreamTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java @@ -18,6 +18,7 @@ package org.apache.doris.tablefunction; import org.apache.doris.analysis.BrokerDesc; +import org.apache.doris.analysis.StorageBackend.StorageType; import org.apache.doris.common.AnalysisException; import org.apache.doris.thrift.TFileType; @@ -29,13 +30,13 @@ /** * The Implement of table valued function - * stream("FORMAT" = "csv"). + * http_stream("FORMAT" = "csv"). */ -public class StreamTableValuedFunction extends ExternalFileTableValuedFunction { - private static final Logger LOG = LogManager.getLogger(StreamTableValuedFunction.class); - public static final String NAME = "stream"; +public class HttpStreamTableValuedFunction extends ExternalFileTableValuedFunction { + private static final Logger LOG = LogManager.getLogger(HttpStreamTableValuedFunction.class); + public static final String NAME = "http_stream"; - public StreamTableValuedFunction(Map params) throws AnalysisException { + public HttpStreamTableValuedFunction(Map params) throws AnalysisException { Map validParams = new CaseInsensitiveMap(); for (String key : params.keySet()) { if (!FILE_FORMAT_PROPERTIES.contains(key.toLowerCase())) { @@ -49,13 +50,7 @@ public StreamTableValuedFunction(Map params) throws AnalysisExce // =========== implement abstract methods of ExternalFileTableValuedFunction ================= @Override public TFileType getTFileType() { - switch (getTFileFormatType()) { - case FORMAT_PARQUET: - case FORMAT_ORC: - return TFileType.FILE_LOCAL; - default: - return TFileType.FILE_STREAM; - } + return TFileType.FILE_STREAM; } @Override @@ -65,12 +60,12 @@ public String getFilePath() { @Override public BrokerDesc getBrokerDesc() { - return null; + return new BrokerDesc("HttpStreamTvfBroker", StorageType.STREAM, locationProperties); } // =========== implement abstract methods of TableValuedFunctionIf ================= @Override public String getTableName() { - return "StreamTableValuedFunction"; + return "HttpStreamTableValuedFunction"; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java index d1b5deeb28f1f3b..2a7b8d3f4f175db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java @@ -50,8 +50,8 @@ public static TableValuedFunctionIf getTableFunction(String funcName, Map resultColumns = readIndexMeta.getSchema(); for (Column column : resultColumns) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/SparkResourceTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/SparkResourceTest.java index 0184acc5c2b93b0..ac047ed17ff00a2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/SparkResourceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/SparkResourceTest.java @@ -36,6 +36,8 @@ import org.junit.Before; import org.junit.Test; +import java.util.Collections; +import java.util.HashMap; import java.util.Map; public class SparkResourceTest { @@ -195,4 +197,39 @@ public void testNoBroker(@Injectable BrokerMgr brokerMgr, @Mocked Env env, stmt.analyze(analyzer); Resource.fromStmt(stmt); } + + @Test + public void testGetEnvConfigsWithoutPrefix() { + Map envConfigs = new HashMap<>(); + envConfigs.put("env.testKey2", "testValue2"); + SparkResource resource = new SparkResource("test", Maps.newHashMap(), null, null, Maps.newHashMap(), + envConfigs) { + @Override + public Map getSystemEnvConfigs() { + return Collections.singletonMap("env.testKey1", "testValue1"); + } + }; + Map expected1 = new HashMap<>(); + expected1.put("testKey1", "testValue1"); + expected1.put("testKey2", "testValue2"); + Map actual1 = resource.getEnvConfigsWithoutPrefix(); + Assert.assertEquals(expected1, actual1); + + Map envConfigs2 = new HashMap<>(); + envConfigs2.put("env.testKey1", "testValue3"); + envConfigs2.put("env.testKey2", "testValue2"); + SparkResource resource2 = new SparkResource("test2", Maps.newHashMap(), null, null, Maps.newHashMap(), + envConfigs2) { + @Override + public Map getSystemEnvConfigs() { + return Collections.singletonMap("env.testKey1", "testValue1"); + } + }; + Map expected2 = new HashMap<>(); + expected2.put("testKey1", "testValue3"); + expected2.put("testKey2", "testValue2"); + Map actual2 = resource2.getEnvConfigsWithoutPrefix(); + Assert.assertEquals(expected2, actual2); + } + } diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java index be377f094ecb602..14cd4772db5292e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java @@ -365,7 +365,7 @@ public void testPendingTaskOnFinishedWithUserInfo(@Mocked BrokerPendingTaskAttac UUID uuid = UUID.randomUUID(); TUniqueId loadId = new TUniqueId(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); Profile jobProfile = new Profile("test", false); - LoadLoadingTask task = new LoadLoadingTask(database, olapTable, brokerDesc, fileGroups, 100, 100, false, 100, + LoadLoadingTask task = new LoadLoadingTask(database, olapTable, brokerDesc, fileGroups, 100, 100, false, false, 100, callback, "", 100, 1, 1, true, jobProfile, false, false, LoadTask.Priority.NORMAL); try { UserIdentity userInfo = new UserIdentity("root", "localhost"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/UnsupportedTypeTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/UnsupportedTypeTest.java index f7a18fda357abfc..20c26286c29e326 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/UnsupportedTypeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/UnsupportedTypeTest.java @@ -62,24 +62,24 @@ public void testUnsupportedTypeThrowException() { "select id from type_tb", "select karr from type_tb", "select array_range(10)", - "select jsonb_parse('{\"k1\":\"v31\",\"k2\":300}')", "select kmap from type_tb1", - "select * from type_tb", "select * from type_tb1", + "select jsonb_parse('{\"k1\":\"v31\",\"k2\":300}')", + "select * from type_tb", }; Class[] exceptions = { null, null, null, - AnalysisException.class, - AnalysisException.class, + null, + null, AnalysisException.class, AnalysisException.class }; - for (int i = 0; i < 3; ++i) { + for (int i = 0; i < 5; ++i) { runPlanner(sqls[i]); } - for (int i = 3; i < sqls.length; ++i) { + for (int i = 5; i < sqls.length; ++i) { int iCopy = i; Assertions.assertThrows(exceptions[i], () -> runPlanner(sqls[iCopy])); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java index fd4d1f2e1287f16..b185c2f5b7f8551 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java @@ -99,7 +99,7 @@ public void testParseCTE() { Assertions.assertEquals(PlanType.LOGICAL_CTE, logicalPlan.getType()); Assertions.assertEquals(((LogicalCTE) logicalPlan).getAliasQueries().size(), 2); - String cteSql3 = "with t1 (key, name) as (select s_suppkey, s_name from supplier) select * from t1"; + String cteSql3 = "with t1 (keyy, name) as (select s_suppkey, s_name from supplier) select * from t1"; logicalPlan = (LogicalPlan) nereidsParser.parseSingle(cteSql3).child(0); Assertions.assertEquals(PlanType.LOGICAL_CTE, logicalPlan.getType()); Assertions.assertEquals(((LogicalCTE) logicalPlan).getAliasQueries().size(), 1); @@ -290,43 +290,43 @@ public void parseSetOperation() { @Test public void testJoinHint() { // no hint - parsePlan("select * from t1 join t2 on t1.key=t2.key") + parsePlan("select * from t1 join t2 on t1.keyy=t2.keyy") .matches(logicalJoin().when(j -> j.getHint() == JoinHint.NONE)); // valid hint - parsePlan("select * from t1 join [shuffle] t2 on t1.key=t2.key") + parsePlan("select * from t1 join [shuffle] t2 on t1.keyy=t2.keyy") .matches(logicalJoin().when(j -> j.getHint() == JoinHint.SHUFFLE_RIGHT)); - parsePlan("select * from t1 join [ shuffle ] t2 on t1.key=t2.key") + parsePlan("select * from t1 join [ shuffle ] t2 on t1.keyy=t2.keyy") .matches(logicalJoin().when(j -> j.getHint() == JoinHint.SHUFFLE_RIGHT)); - parsePlan("select * from t1 join [broadcast] t2 on t1.key=t2.key") + parsePlan("select * from t1 join [broadcast] t2 on t1.keyy=t2.keyy") .matches(logicalJoin().when(j -> j.getHint() == JoinHint.BROADCAST_RIGHT)); - parsePlan("select * from t1 join /*+ broadcast */ t2 on t1.key=t2.key") + parsePlan("select * from t1 join /*+ broadcast */ t2 on t1.keyy=t2.keyy") .matches(logicalJoin().when(j -> j.getHint() == JoinHint.BROADCAST_RIGHT)); // invalid hint position - parsePlan("select * from [shuffle] t1 join t2 on t1.key=t2.key") + parsePlan("select * from [shuffle] t1 join t2 on t1.keyy=t2.keyy") .assertThrowsExactly(ParseException.class); - parsePlan("select * from /*+ shuffle */ t1 join t2 on t1.key=t2.key") + parsePlan("select * from /*+ shuffle */ t1 join t2 on t1.keyy=t2.keyy") .assertThrowsExactly(ParseException.class); // invalid hint content - parsePlan("select * from t1 join [bucket] t2 on t1.key=t2.key") + parsePlan("select * from t1 join [bucket] t2 on t1.keyy=t2.keyy") .assertThrowsExactly(ParseException.class) .assertMessageContains("Invalid join hint: bucket(line 1, pos 22)\n" + "\n" + "== SQL ==\n" - + "select * from t1 join [bucket] t2 on t1.key=t2.key\n" + + "select * from t1 join [bucket] t2 on t1.keyy=t2.keyy\n" + "----------------------^^^"); // invalid multiple hints - parsePlan("select * from t1 join /*+ shuffle , broadcast */ t2 on t1.key=t2.key") + parsePlan("select * from t1 join /*+ shuffle , broadcast */ t2 on t1.keyy=t2.keyy") .assertThrowsExactly(ParseException.class); - parsePlan("select * from t1 join [shuffle,broadcast] t2 on t1.key=t2.key") + parsePlan("select * from t1 join [shuffle,broadcast] t2 on t1.keyy=t2.keyy") .assertThrowsExactly(ParseException.class); } @@ -346,4 +346,23 @@ public void testParseCast() { Assertions.assertEquals(6, decimalV2Type.getScale()); } } + + @Test + + void testParseExprDepthWidth() { + String sql = "SELECT 1+2 = 3 from t"; + NereidsParser nereidsParser = new NereidsParser(); + LogicalPlan logicalPlan = (LogicalPlan) nereidsParser.parseSingle(sql).child(0); + System.out.println(logicalPlan); + // alias (1 + 2 = 3) + Assertions.assertEquals(4, logicalPlan.getExpressions().get(0).getDepth()); + Assertions.assertEquals(3, logicalPlan.getExpressions().get(0).getWidth()); + } + + @Test + public void testParseCollate() { + String sql = "SELECT * FROM t1 WHERE col COLLATE utf8 = 'test'"; + NereidsParser nereidsParser = new NereidsParser(); + nereidsParser.parseSingle(sql); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java index 9c9634e44d4f9fe..be0f64dba9b6eb8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.ColocateTableIndex; import org.apache.doris.catalog.Env; import org.apache.doris.common.Pair; +import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; import org.apache.doris.nereids.trees.expressions.AssertNumRowsElement; @@ -67,8 +68,7 @@ import java.util.Optional; import java.util.stream.Collectors; -@SuppressWarnings("unused") -public class ChildOutputPropertyDeriverTest { +class ChildOutputPropertyDeriverTest { @Mocked GroupPlan groupPlan; @@ -97,11 +97,12 @@ ConnectContext get() { } @Test - public void testInnerJoin() { + void testInnerJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.INNER_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -135,11 +136,12 @@ public void testInnerJoin() { } @Test - public void testCrossJoin() { + void testCrossJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.CROSS_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -173,11 +175,12 @@ public void testCrossJoin() { } @Test - public void testLeftOuterJoin() { + void testLeftOuterJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.LEFT_OUTER_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -212,11 +215,12 @@ public void testLeftOuterJoin() { } @Test - public void testLeftSemiJoin() { + void testLeftSemiJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.LEFT_SEMI_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -251,11 +255,12 @@ public void testLeftSemiJoin() { } @Test - public void testLeftAntiJoin() { + void testLeftAntiJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.LEFT_ANTI_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -290,11 +295,12 @@ public void testLeftAntiJoin() { } @Test - public void testNullAwareLeftAntiJoin() { + void testNullAwareLeftAntiJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.NULL_AWARE_LEFT_ANTI_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -329,11 +335,12 @@ public void testNullAwareLeftAntiJoin() { } @Test - public void testRightSemiJoin() { + void testRightSemiJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.RIGHT_SEMI_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -369,11 +376,12 @@ public void testRightSemiJoin() { } @Test - public void testRightAntiJoin() { + void testRightAntiJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.RIGHT_ANTI_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -409,11 +417,12 @@ public void testRightAntiJoin() { } @Test - public void testRightOuterJoin() { + void testRightOuterJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.RIGHT_OUTER_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -449,11 +458,12 @@ public void testRightOuterJoin() { } @Test - public void testFullOuterJoin() { + void testFullOuterJoin() { PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.FULL_OUTER_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); PhysicalProperties left = new PhysicalProperties( new DistributionSpecHash( @@ -483,7 +493,7 @@ public void testFullOuterJoin() { } @Test - public void testBroadcastJoin() { + void testBroadcastJoin() { new MockUp() { @Mock Pair, List> getOnClauseUsedSlots( @@ -499,6 +509,7 @@ Pair, List> getOnClauseUsedSlots( Collections.emptyList()))), ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); Map leftMap = Maps.newHashMap(); leftMap.put(new ExprId(0), 0); @@ -529,7 +540,7 @@ Pair, List> getOnClauseUsedSlots( } @Test - public void testShuffleJoin() { + void testShuffleJoin() { new MockUp() { @Mock Pair, List> getOnClauseUsedSlots( @@ -545,6 +556,7 @@ Pair, List> getOnClauseUsedSlots( Collections.emptyList()))), ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); Map leftMap = Maps.newHashMap(); leftMap.put(new ExprId(0), 0); @@ -578,11 +590,12 @@ Pair, List> getOnClauseUsedSlots( } @Test - public void testNestedLoopJoin() { + void testNestedLoopJoin() { PhysicalNestedLoopJoin join = new PhysicalNestedLoopJoin<>(JoinType.CROSS_JOIN, ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, Optional.empty(), logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); Map leftMap = Maps.newHashMap(); leftMap.put(new ExprId(0), 0); @@ -608,7 +621,7 @@ public void testNestedLoopJoin() { } @Test - public void testLocalPhaseAggregate() { + void testLocalPhaseAggregate() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); PhysicalHashAggregate aggregate = new PhysicalHashAggregate<>( Lists.newArrayList(key), @@ -620,6 +633,7 @@ public void testLocalPhaseAggregate() { groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); + new Group(null, groupExpression, null); PhysicalProperties child = new PhysicalProperties(DistributionSpecReplicated.INSTANCE, new OrderSpec(Lists.newArrayList( new OrderKey(new SlotReference("ignored", IntegerType.INSTANCE), true, true)))); @@ -631,7 +645,7 @@ public void testLocalPhaseAggregate() { } @Test - public void testGlobalPhaseAggregate() { + void testGlobalPhaseAggregate() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); SlotReference partition = new SlotReference("col2", BigIntType.INSTANCE); PhysicalHashAggregate aggregate = new PhysicalHashAggregate<>( @@ -644,6 +658,7 @@ public void testGlobalPhaseAggregate() { groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); + new Group(null, groupExpression, null); DistributionSpecHash childHash = new DistributionSpecHash(Lists.newArrayList(partition.getExprId()), ShuffleType.EXECUTION_BUCKETED); PhysicalProperties child = new PhysicalProperties(childHash, @@ -662,7 +677,7 @@ public void testGlobalPhaseAggregate() { } @Test - public void testAggregateWithoutGroupBy() { + void testAggregateWithoutGroupBy() { PhysicalHashAggregate aggregate = new PhysicalHashAggregate<>( Lists.newArrayList(), Lists.newArrayList(), @@ -674,6 +689,7 @@ public void testAggregateWithoutGroupBy() { ); GroupExpression groupExpression = new GroupExpression(aggregate); + new Group(null, groupExpression, null); PhysicalProperties child = new PhysicalProperties(DistributionSpecGather.INSTANCE, new OrderSpec(Lists.newArrayList( new OrderKey(new SlotReference("ignored", IntegerType.INSTANCE), true, true)))); @@ -684,11 +700,12 @@ public void testAggregateWithoutGroupBy() { } @Test - public void testLocalQuickSort() { + void testLocalQuickSort() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); List orderKeys = Lists.newArrayList(new OrderKey(key, true, true)); PhysicalQuickSort sort = new PhysicalQuickSort<>(orderKeys, SortPhase.LOCAL_SORT, logicalProperties, groupPlan); GroupExpression groupExpression = new GroupExpression(sort); + new Group(null, groupExpression, null); PhysicalProperties child = new PhysicalProperties(DistributionSpecReplicated.INSTANCE, new OrderSpec(Lists.newArrayList( new OrderKey(new SlotReference("ignored", IntegerType.INSTANCE), true, true)))); @@ -700,11 +717,12 @@ public void testLocalQuickSort() { } @Test - public void testQuickSort() { + void testQuickSort() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); List orderKeys = Lists.newArrayList(new OrderKey(key, true, true)); PhysicalQuickSort sort = new PhysicalQuickSort<>(orderKeys, SortPhase.MERGE_SORT, logicalProperties, groupPlan); GroupExpression groupExpression = new GroupExpression(sort); + new Group(null, groupExpression, null); PhysicalProperties child = new PhysicalProperties(DistributionSpecReplicated.INSTANCE, new OrderSpec(Lists.newArrayList( new OrderKey(new SlotReference("ignored", IntegerType.INSTANCE), true, true)))); @@ -716,12 +734,13 @@ public void testQuickSort() { } @Test - public void testTopN() { + void testTopN() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); List orderKeys = Lists.newArrayList(new OrderKey(key, true, true)); // localSort require any PhysicalTopN sort = new PhysicalTopN<>(orderKeys, 10, 10, SortPhase.LOCAL_SORT, logicalProperties, groupPlan); GroupExpression groupExpression = new GroupExpression(sort); + new Group(null, groupExpression, null); PhysicalProperties child = new PhysicalProperties(DistributionSpecReplicated.INSTANCE, new OrderSpec(Lists.newArrayList( new OrderKey(new SlotReference("ignored", IntegerType.INSTANCE), true, true)))); @@ -733,6 +752,7 @@ public void testTopN() { // merge/gather sort requires gather sort = new PhysicalTopN<>(orderKeys, 10, 10, SortPhase.MERGE_SORT, logicalProperties, groupPlan); groupExpression = new GroupExpression(sort); + new Group(null, groupExpression, null); child = new PhysicalProperties(DistributionSpecReplicated.INSTANCE, new OrderSpec(Lists.newArrayList( new OrderKey(new SlotReference("ignored", IntegerType.INSTANCE), true, true)))); @@ -744,11 +764,12 @@ public void testTopN() { } @Test - public void testLimit() { + void testLimit() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); List orderKeys = Lists.newArrayList(new OrderKey(key, true, true)); PhysicalLimit limit = new PhysicalLimit<>(10, 10, LimitPhase.ORIGIN, logicalProperties, groupPlan); GroupExpression groupExpression = new GroupExpression(limit); + new Group(null, groupExpression, null); PhysicalProperties child = new PhysicalProperties(DistributionSpecGather.INSTANCE, new OrderSpec(orderKeys)); @@ -759,13 +780,14 @@ public void testLimit() { } @Test - public void testAssertNumRows() { + void testAssertNumRows() { PhysicalAssertNumRows assertNumRows = new PhysicalAssertNumRows<>( new AssertNumRowsElement(1, "", AssertNumRowsElement.Assertion.EQ), logicalProperties, groupPlan ); GroupExpression groupExpression = new GroupExpression(assertNumRows); + new Group(null, groupExpression, null); PhysicalProperties child = new PhysicalProperties(DistributionSpecGather.INSTANCE, new OrderSpec()); ChildOutputPropertyDeriver deriver = new ChildOutputPropertyDeriver(Lists.newArrayList(child)); PhysicalProperties result = deriver.getOutputProperties(groupExpression); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java index 1df7067c2b088d9..4ccee56e4b1c929 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java @@ -19,6 +19,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; import org.apache.doris.nereids.trees.expressions.AssertNumRowsElement; @@ -51,8 +52,7 @@ import java.util.List; import java.util.Optional; -@SuppressWarnings("unused") -public class RequestPropertyDeriverTest { +class RequestPropertyDeriverTest { @Mocked GroupPlan groupPlan; @@ -75,11 +75,13 @@ public void setUp() { } @Test - public void testNestedLoopJoin() { + void testNestedLoopJoin() { PhysicalNestedLoopJoin join = new PhysicalNestedLoopJoin<>(JoinType.CROSS_JOIN, - ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, Optional.empty(), logicalProperties, groupPlan, + ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, Optional.empty(), logicalProperties, + groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); RequestPropertyDeriver requestPropertyDeriver = new RequestPropertyDeriver(jobContext); List> actual @@ -91,7 +93,7 @@ public void testNestedLoopJoin() { } @Test - public void testShuffleHashJoin() { + void testShuffleHashJoin() { new MockUp() { @Mock Pair, List> getHashConjunctsExprIds() { @@ -100,9 +102,11 @@ Pair, List> getHashConjunctsExprIds() { }; PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.RIGHT_OUTER_JOIN, - ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, + ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), + logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); RequestPropertyDeriver requestPropertyDeriver = new RequestPropertyDeriver(jobContext); List> actual @@ -110,14 +114,15 @@ Pair, List> getHashConjunctsExprIds() { List> expected = Lists.newArrayList(); expected.add(Lists.newArrayList( - new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(0)), ShuffleType.REQUIRE)), + new PhysicalProperties( + new DistributionSpecHash(Lists.newArrayList(new ExprId(0)), ShuffleType.REQUIRE)), new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(1)), ShuffleType.REQUIRE)) )); Assertions.assertEquals(expected, actual); } @Test - public void testShuffleOrBroadcastHashJoin() { + void testShuffleOrBroadcastHashJoin() { new MockUp() { @Mock Pair, List> getHashConjunctsExprIds() { @@ -126,9 +131,11 @@ Pair, List> getHashConjunctsExprIds() { }; PhysicalHashJoin join = new PhysicalHashJoin<>(JoinType.INNER_JOIN, - ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), logicalProperties, + ExpressionUtils.EMPTY_CONDITION, ExpressionUtils.EMPTY_CONDITION, JoinHint.NONE, Optional.empty(), + logicalProperties, groupPlan, groupPlan); GroupExpression groupExpression = new GroupExpression(join); + new Group(null, groupExpression, null); RequestPropertyDeriver requestPropertyDeriver = new RequestPropertyDeriver(jobContext); List> actual @@ -136,7 +143,8 @@ Pair, List> getHashConjunctsExprIds() { List> expected = Lists.newArrayList(); expected.add(Lists.newArrayList( - new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(0)), ShuffleType.REQUIRE)), + new PhysicalProperties( + new DistributionSpecHash(Lists.newArrayList(new ExprId(0)), ShuffleType.REQUIRE)), new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(1)), ShuffleType.REQUIRE)) )); expected.add(Lists.newArrayList(PhysicalProperties.ANY, PhysicalProperties.REPLICATED)); @@ -144,7 +152,7 @@ Pair, List> getHashConjunctsExprIds() { } @Test - public void testLocalAggregate() { + void testLocalAggregate() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); PhysicalHashAggregate aggregate = new PhysicalHashAggregate<>( Lists.newArrayList(key), @@ -156,6 +164,7 @@ public void testLocalAggregate() { groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); + new Group(null, groupExpression, null); RequestPropertyDeriver requestPropertyDeriver = new RequestPropertyDeriver(jobContext); List> actual = requestPropertyDeriver.getRequestChildrenPropertyList(groupExpression); @@ -165,7 +174,7 @@ public void testLocalAggregate() { } @Test - public void testGlobalAggregate() { + void testGlobalAggregate() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); SlotReference partition = new SlotReference("partition", IntegerType.INSTANCE); PhysicalHashAggregate aggregate = new PhysicalHashAggregate<>( @@ -178,6 +187,7 @@ public void testGlobalAggregate() { groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); + new Group(null, groupExpression, null); RequestPropertyDeriver requestPropertyDeriver = new RequestPropertyDeriver(jobContext); List> actual = requestPropertyDeriver.getRequestChildrenPropertyList(groupExpression); @@ -190,7 +200,7 @@ public void testGlobalAggregate() { } @Test - public void testGlobalAggregateWithoutPartition() { + void testGlobalAggregateWithoutPartition() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); PhysicalHashAggregate aggregate = new PhysicalHashAggregate<>( Lists.newArrayList(), @@ -202,6 +212,7 @@ public void testGlobalAggregateWithoutPartition() { groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); + new Group(null, groupExpression, null); RequestPropertyDeriver requestPropertyDeriver = new RequestPropertyDeriver(jobContext); List> actual = requestPropertyDeriver.getRequestChildrenPropertyList(groupExpression); @@ -211,13 +222,14 @@ public void testGlobalAggregateWithoutPartition() { } @Test - public void testAssertNumRows() { + void testAssertNumRows() { PhysicalAssertNumRows assertNumRows = new PhysicalAssertNumRows<>( new AssertNumRowsElement(1, "", AssertNumRowsElement.Assertion.EQ), logicalProperties, groupPlan ); GroupExpression groupExpression = new GroupExpression(assertNumRows); + new Group(null, groupExpression, null); RequestPropertyDeriver requestPropertyDeriver = new RequestPropertyDeriver(jobContext); List> actual = requestPropertyDeriver.getRequestChildrenPropertyList(groupExpression); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java index 522f198e3ff774d..ef5a32e2d3bcdbc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java @@ -140,7 +140,7 @@ public void testCTEInHavingAndSubquery() { logicalFilter( logicalProject( logicalJoin( - logicalAggregate(), + logicalProject(logicalAggregate()), logicalProject() ) ) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java index bf060d7e5dd8a43..73422bee7024002 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java @@ -156,18 +156,20 @@ public void testWhereSql2AfterAnalyzed() { .matchesNotCheck( logicalApply( any(), - logicalAggregate( - logicalFilter() - ).when(FieldChecker.check("outputExpressions", ImmutableList.of( - new Alias(new ExprId(7), - (new Sum( - new SlotReference(new ExprId(4), "k3", - BigIntType.INSTANCE, true, - ImmutableList.of( - "default_cluster:test", - "t7")))).withAlwaysNullable( - true), - "sum(k3)")))) + logicalProject( + logicalAggregate( + logicalProject() + ).when(FieldChecker.check("outputExpressions", ImmutableList.of( + new Alias(new ExprId(7), + (new Sum( + new SlotReference(new ExprId(4), "k3", + BigIntType.INSTANCE, true, + ImmutableList.of( + "default_cluster:test", + "t7")))).withAlwaysNullable( + true), + "sum(k3)")))) + ) ).when(FieldChecker.check("correlationSlot", ImmutableList.of( new SlotReference(new ExprId(1), "k2", BigIntType.INSTANCE, true, ImmutableList.of("default_cluster:test", "t6")) @@ -383,28 +385,32 @@ public void testSql10AfterAnalyze() { logicalProject( logicalApply( any(), - logicalAggregate( - logicalSubQueryAlias( + logicalProject( + logicalAggregate( logicalProject( - logicalFilter() - ).when(p -> p.getProjects().equals(ImmutableList.of( - new Alias(new ExprId(7), new SlotReference(new ExprId(5), "v1", BigIntType.INSTANCE, - true, - ImmutableList.of("default_cluster:test", "t7")), "aa") - ))) - ) - .when(a -> a.getAlias().equals("t2")) - .when(a -> a.getOutput().equals(ImmutableList.of( - new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, - true, ImmutableList.of("t2")) + logicalSubQueryAlias( + logicalProject( + logicalFilter() + ).when(p -> p.getProjects().equals(ImmutableList.of( + new Alias(new ExprId(7), new SlotReference(new ExprId(5), "v1", BigIntType.INSTANCE, + true, + ImmutableList.of("default_cluster:test", "t7")), "aa") + ))) + ) + .when(a -> a.getAlias().equals("t2")) + .when(a -> a.getOutput().equals(ImmutableList.of( + new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, + true, ImmutableList.of("t2")) + ))) + ) + ).when(agg -> agg.getOutputExpressions().equals(ImmutableList.of( + new Alias(new ExprId(8), + (new Max(new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, + true, + ImmutableList.of("t2")))).withAlwaysNullable(true), "max(aa)") ))) - ).when(agg -> agg.getOutputExpressions().equals(ImmutableList.of( - new Alias(new ExprId(8), - (new Max(new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, - true, - ImmutableList.of("t2")))).withAlwaysNullable(true), "max(aa)") - ))) - .when(agg -> agg.getGroupByExpressions().equals(ImmutableList.of())) + .when(agg -> agg.getGroupByExpressions().equals(ImmutableList.of())) + ) ) .when(apply -> apply.getCorrelationSlot().equals(ImmutableList.of( new SlotReference(new ExprId(1), "k2", BigIntType.INSTANCE, true, diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java index 0a3334b4cf0c2b1..dc05ec062637ea1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java @@ -90,10 +90,11 @@ public void testGroupByOnJoin() { join ); PlanChecker checker = PlanChecker.from(MemoTestUtils.createConnectContext()).analyze(aggregate); - LogicalAggregate plan = (LogicalAggregate) checker.getCascadesContext().getMemo().copyOut(); + LogicalAggregate plan = (LogicalAggregate) ((LogicalProject) checker.getCascadesContext() + .getMemo().copyOut()).child(); SlotReference groupByKey = (SlotReference) plan.getGroupByExpressions().get(0); - SlotReference t1id = (SlotReference) ((LogicalJoin) plan.child()).left().getOutput().get(0); - SlotReference t2id = (SlotReference) ((LogicalJoin) plan.child()).right().getOutput().get(0); + SlotReference t1id = (SlotReference) ((LogicalJoin) plan.child().child(0)).left().getOutput().get(0); + SlotReference t2id = (SlotReference) ((LogicalJoin) plan.child().child(0)).right().getOutput().get(0); Assertions.assertEquals(groupByKey.getExprId(), t1id.getExprId()); Assertions.assertNotEquals(t1id.getExprId(), t2id.getExprId()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstantTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstantTest.java similarity index 98% rename from fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstantTest.java rename to fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstantTest.java index 3fca54eed96a294..c35b983911c8598 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstantTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstantTest.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.nereids.rules.rewrite; +package org.apache.doris.nereids.rules.analysis; import org.apache.doris.catalog.AggregateType; import org.apache.doris.catalog.Column; @@ -23,7 +23,6 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.Type; -import org.apache.doris.nereids.rules.analysis.CheckAfterRewrite; import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Slot; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java index 03cc549bc2c1faa..8cacb4609186c32 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java @@ -35,6 +35,7 @@ import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.nereids.types.TinyIntType; import org.apache.doris.nereids.util.FieldChecker; import org.apache.doris.nereids.util.MemoPatternMatchSupported; @@ -45,8 +46,6 @@ import com.google.common.collect.Lists; import org.junit.jupiter.api.Test; -import java.util.stream.Collectors; - public class FillUpMissingSlotsTest extends AnalyzeCheckTestBase implements MemoPatternMatchSupported { @Override @@ -86,35 +85,35 @@ public void testHavingGroupBySlot() { ); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1))))); + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1)))))); sql = "SELECT a1 as value FROM t1 GROUP BY a1 HAVING a1 > 0"; - a1 = new SlotReference( - new ExprId(1), "a1", TinyIntType.INSTANCE, true, - ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1") - ); - Alias value = new Alias(new ExprId(3), a1, "value"); + SlotReference value = new SlotReference(new ExprId(3), "value", TinyIntType.INSTANCE, true, + ImmutableList.of()); PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new ExpressionRewrite(FunctionBinder.INSTANCE)) .matches( - logicalProject( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(value))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(value.toSlot(), new TinyIntLiteral((byte) 0))))))); + logicalProject( + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(value)))) + ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(value.toSlot(), new TinyIntLiteral((byte) 0))))))); sql = "SELECT a1 as value FROM t1 GROUP BY a1 HAVING value > 0"; PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new ExpressionRewrite(FunctionBinder.INSTANCE)) .matches( logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(value))) + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(value)))) ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(value.toSlot(), new TinyIntLiteral((byte) 0)))))); sql = "SELECT SUM(a2) FROM t1 GROUP BY a1 HAVING a1 > 0"; @@ -130,13 +129,14 @@ public void testHavingGroupBySlot() { PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new ExpressionRewrite(FunctionBinder.INSTANCE)) .matches( - logicalProject( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(sumA2, a1))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(a1, new TinyIntLiteral((byte) 0))))) - ).when(FieldChecker.check("projects", Lists.newArrayList(sumA2.toSlot())))); + logicalProject( + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2)))) + ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(a1, new TinyIntLiteral((byte) 0))))) + ).when(FieldChecker.check("projects", Lists.newArrayList(sumA2.toSlot())))); } @Test @@ -153,24 +153,28 @@ public void testHavingAggregateFunction() { Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "sum(a2)"); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalProject( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(sumA2.toSlot(), Literal.of(0L))))) - ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot())))); + logicalProject( + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2)))) + ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(sumA2.toSlot(), Literal.of(0L))))) + ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot())))); sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 HAVING SUM(a2) > 0"; sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)"); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalProject( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(sumA2.toSlot(), Literal.of(0L))))))); + logicalProject( + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject( + logicalOlapScan() + ) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2)))) + ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(sumA2.toSlot(), Literal.of(0L))))))); sql = "SELECT a1, SUM(a2) as value FROM t1 GROUP BY a1 HAVING SUM(a2) > 0"; a1 = new SlotReference( @@ -184,20 +188,24 @@ public void testHavingAggregateFunction() { Alias value = new Alias(new ExprId(3), new Sum(a2), "value"); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalProject( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, value))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(value.toSlot(), Literal.of(0L))))))); + logicalProject( + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject( + logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, value)))) + ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(value.toSlot(), Literal.of(0L))))))); sql = "SELECT a1, SUM(a2) as value FROM t1 GROUP BY a1 HAVING value > 0"; PlanChecker.from(connectContext).analyze(sql) .matches( logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, value))) + logicalProject( + logicalAggregate( + logicalProject( + logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, value)))) ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(value.toSlot(), Literal.of(0L)))))); sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 HAVING MIN(pk) > 0"; @@ -217,49 +225,53 @@ public void testHavingAggregateFunction() { Alias minPK = new Alias(new ExprId(4), new Min(pk), "min(pk)"); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalProject( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2, minPK))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(minPK.toSlot(), Literal.of((byte) 0))))) - ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA2.toSlot())))); + logicalProject( + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2, minPK)))) + ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(minPK.toSlot(), Literal.of((byte) 0))))) + ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA2.toSlot())))); sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 HAVING SUM(a1 + a2) > 0"; Alias sumA1A2 = new Alias(new ExprId(3), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalProject( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(sumA1A2.toSlot(), Literal.of(0L))))))); + logicalProject( + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2)))) + ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(sumA1A2.toSlot(), Literal.of(0L))))))); sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 HAVING SUM(a1 + a2 + 3) > 0"; Alias sumA1A23 = new Alias(new ExprId(4), new Sum(new Add(new Add(a1, a2), new TinyIntLiteral((byte) 3))), "sum(((a1 + a2) + 3))"); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalProject( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2, sumA1A23))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(sumA1A23.toSlot(), Literal.of(0L))))) - ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA1A2.toSlot())))); + logicalProject( + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2, sumA1A23)))) + ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(sumA1A23.toSlot(), Literal.of(0L))))) + ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA1A2.toSlot())))); sql = "SELECT a1 FROM t1 GROUP BY a1 HAVING COUNT(*) > 0"; Alias countStar = new Alias(new ExprId(3), new Count(), "count(*)"); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalProject( - logicalFilter( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, countStar))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(countStar.toSlot(), Literal.of(0L))))) - ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot())))); + logicalProject( + logicalFilter( + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, countStar)))) + ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(countStar.toSlot(), Literal.of(0L))))) + ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot())))); } @Test @@ -281,19 +293,21 @@ void testJoinWithHaving() { Alias sumB1 = new Alias(new ExprId(7), new Sum(b1), "sum(b1)"); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalProject( - logicalFilter( - logicalAggregate( + logicalProject( logicalFilter( - logicalJoin( - logicalOlapScan(), - logicalOlapScan() - ) - ) - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2, sumB1))) - ).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(new Cast(a1, BigIntType.INSTANCE), - sumB1.toSlot())))) - ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA2.toSlot())))); + logicalProject( + logicalAggregate( + logicalProject( + logicalFilter( + logicalJoin( + logicalOlapScan(), + logicalOlapScan() + ) + )) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2, sumB1))) + )).when(FieldChecker.check("conjuncts", ImmutableSet.of(new GreaterThan(new Cast(a1, BigIntType.INSTANCE), + sumB1.toSlot())))) + ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA2.toSlot())))); } @Test @@ -331,6 +345,10 @@ void testComplexQueryWithHaving() { new ExprId(0), "pk", TinyIntType.INSTANCE, true, ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1") ); + SlotReference pk1 = new SlotReference( + new ExprId(6), "(pk + 1)", IntegerType.INSTANCE, true, + ImmutableList.of() + ); SlotReference a1 = new SlotReference( new ExprId(1), "a1", TinyIntType.INSTANCE, true, ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1") @@ -339,40 +357,42 @@ void testComplexQueryWithHaving() { new ExprId(2), "a2", TinyIntType.INSTANCE, true, ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1") ); - Alias pk1 = new Alias(new ExprId(6), new Add(pk, Literal.of((byte) 1)), "(pk + 1)"); Alias pk11 = new Alias(new ExprId(7), new Add(new Add(pk, Literal.of((byte) 1)), Literal.of((byte) 1)), "((pk + 1) + 1)"); Alias pk2 = new Alias(new ExprId(8), new Add(pk, Literal.of((byte) 2)), "(pk + 2)"); Alias sumA1 = new Alias(new ExprId(9), new Sum(a1), "SUM(a1)"); - Alias countA11 = new Alias(new ExprId(10), new Add(new Count(a1), Literal.of((byte) 1)), "(COUNT(a1) + 1)"); + Alias countA1 = new Alias(new ExprId(13), new Count(a1), "count(a1)"); + Alias countA11 = new Alias(new ExprId(10), new Add(countA1.toSlot(), Literal.of((byte) 1)), "(COUNT(a1) + 1)"); Alias sumA1A2 = new Alias(new ExprId(11), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); Alias v1 = new Alias(new ExprId(12), new Count(a2), "v1"); PlanChecker.from(connectContext).analyze(sql) .matches( - logicalProject( - logicalFilter( - logicalAggregate( + logicalProject( logicalFilter( - logicalJoin( - logicalOlapScan(), - logicalOlapScan() - ) + logicalProject( + logicalAggregate( + logicalProject( + logicalFilter( + logicalJoin( + logicalOlapScan(), + logicalOlapScan() + ) + )) + ).when(FieldChecker.check("outputExpressions", + Lists.newArrayList(pk, pk1, sumA1, countA1, sumA1A2, v1)))) + ).when(FieldChecker.check("conjuncts", + ImmutableSet.of( + new GreaterThan(pk.toSlot(), Literal.of((byte) 0)), + new GreaterThan(countA11.toSlot(), Literal.of(0L)), + new GreaterThan(new Add(sumA1A2.toSlot(), Literal.of((byte) 1)), Literal.of(0L)), + new GreaterThan(new Add(v1.toSlot(), Literal.of((byte) 1)), Literal.of(0L)), + new GreaterThan(v1.toSlot(), Literal.of(0L)) + )) ) - ).when(FieldChecker.check("outputExpressions", - Lists.newArrayList(pk1, pk11, pk2, sumA1, countA11, sumA1A2, v1, pk))) - ).when(FieldChecker.check("conjuncts", - ImmutableSet.of( - new GreaterThan(pk.toSlot(), Literal.of((byte) 0)), - new GreaterThan(countA11.toSlot(), Literal.of(0L)), - new GreaterThan(new Add(sumA1A2.toSlot(), Literal.of((byte) 1)), Literal.of(0L)), - new GreaterThan(new Add(v1.toSlot(), Literal.of((byte) 1)), Literal.of(0L)), - new GreaterThan(v1.toSlot(), Literal.of(0L)) - )) - ) - ).when(FieldChecker.check( - "projects", Lists.newArrayList( - pk1, pk11, pk2, sumA1, countA11, sumA1A2, v1).stream() - .map(Alias::toSlot).collect(Collectors.toList())) - )); + ).when(FieldChecker.check( + "projects", Lists.newArrayList( + pk1, pk11.toSlot(), pk2.toSlot(), sumA1.toSlot(), countA11.toSlot(), sumA1A2.toSlot(), v1.toSlot()) + ) + )); } @Test @@ -391,9 +411,10 @@ public void testSortAggregateFunction() { .matches( logicalProject( logicalSort( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2))) + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2)))) ).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA2.toSlot(), true, true)))) ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot())))); @@ -402,9 +423,10 @@ public void testSortAggregateFunction() { PlanChecker.from(connectContext).analyze(sql) .matches( logicalSort( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2))) + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2)))) ).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA2.toSlot(), true, true))))); sql = "SELECT a1, SUM(a2) as value FROM t1 GROUP BY a1 ORDER BY SUM(a2)"; @@ -420,9 +442,10 @@ public void testSortAggregateFunction() { PlanChecker.from(connectContext).analyze(sql) .matches( logicalSort( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, value))) + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, value)))) ).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA2.toSlot(), true, true))))); sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 ORDER BY MIN(pk)"; @@ -444,9 +467,10 @@ public void testSortAggregateFunction() { .matches( logicalProject( logicalSort( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2, minPK))) + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2, minPK)))) ).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(minPK.toSlot(), true, true)))) ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA2.toSlot())))); @@ -455,9 +479,10 @@ public void testSortAggregateFunction() { PlanChecker.from(connectContext).analyze(sql) .matches( logicalSort( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2))) + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2)))) ).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA1A2.toSlot(), true, true))))); sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 ORDER BY SUM(a1 + a2 + 3)"; @@ -467,9 +492,10 @@ public void testSortAggregateFunction() { .matches( logicalProject( logicalSort( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2, sumA1A23))) + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2, sumA1A23)))) ).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA1A23.toSlot(), true, true)))) ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA1A2.toSlot())))); @@ -479,9 +505,10 @@ public void testSortAggregateFunction() { .matches( logicalProject( logicalSort( - logicalAggregate( - logicalOlapScan() - ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, countStar))) + logicalProject( + logicalAggregate( + logicalProject(logicalOlapScan()) + ).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, countStar)))) ).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(countStar.toSlot(), true, true)))) ).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot())))); } @@ -495,6 +522,10 @@ void testComplexQueryWithOrderBy() { new ExprId(0), "pk", TinyIntType.INSTANCE, true, ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1") ); + SlotReference pk1 = new SlotReference( + new ExprId(6), "(pk + 1)", IntegerType.INSTANCE, true, + ImmutableList.of() + ); SlotReference a1 = new SlotReference( new ExprId(1), "a1", TinyIntType.INSTANCE, true, ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1") @@ -503,40 +534,41 @@ void testComplexQueryWithOrderBy() { new ExprId(2), "a2", TinyIntType.INSTANCE, true, ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1") ); - Alias pk1 = new Alias(new ExprId(6), new Add(pk, Literal.of((byte) 1)), "(pk + 1)"); Alias pk11 = new Alias(new ExprId(7), new Add(new Add(pk, Literal.of((byte) 1)), Literal.of((byte) 1)), "((pk + 1) + 1)"); Alias pk2 = new Alias(new ExprId(8), new Add(pk, Literal.of((byte) 2)), "(pk + 2)"); Alias sumA1 = new Alias(new ExprId(9), new Sum(a1), "SUM(a1)"); + Alias countA1 = new Alias(new ExprId(13), new Count(a1), "count(a1)"); Alias countA11 = new Alias(new ExprId(10), new Add(new Count(a1), Literal.of((byte) 1)), "(COUNT(a1) + 1)"); Alias sumA1A2 = new Alias(new ExprId(11), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); Alias v1 = new Alias(new ExprId(12), new Count(a2), "v1"); PlanChecker.from(connectContext).analyze(sql) - .matches( - logicalProject( - logicalSort( - logicalAggregate( - logicalFilter( - logicalJoin( - logicalOlapScan(), - logicalOlapScan() - ) - ) - ).when(FieldChecker.check("outputExpressions", - Lists.newArrayList(pk1, pk11, pk2, sumA1, countA11, sumA1A2, v1, pk))) - ).when(FieldChecker.check("orderKeys", - ImmutableList.of( - new OrderKey(pk, true, true), - new OrderKey(countA11.toSlot(), true, true), - new OrderKey(new Add(sumA1A2.toSlot(), new TinyIntLiteral((byte) 1)), true, true), - new OrderKey(new Add(v1.toSlot(), new TinyIntLiteral((byte) 1)), true, true), - new OrderKey(v1.toSlot(), true, true) - ) - )) - ).when(FieldChecker.check( - "projects", Lists.newArrayList( - pk1, pk11, pk2, sumA1, countA11, sumA1A2, v1).stream() - .map(Alias::toSlot).collect(Collectors.toList())) - )); + .matches(logicalProject(logicalSort(logicalProject(logicalAggregate(logicalProject( + logicalFilter(logicalJoin(logicalOlapScan(), logicalOlapScan())))).when( + FieldChecker.check("outputExpressions", Lists.newArrayList(pk, pk1, + sumA1, countA1, sumA1A2, v1))))).when(FieldChecker.check( + "orderKeys", + ImmutableList.of(new OrderKey(pk, true, true), + new OrderKey( + countA11.toSlot(), true, true), + new OrderKey( + new Add(sumA1A2.toSlot(), + new TinyIntLiteral( + (byte) 1)), + true, true), + new OrderKey( + new Add(v1.toSlot(), + new TinyIntLiteral( + (byte) 1)), + true, true), + new OrderKey(v1.toSlot(), true, true))))) + .when(FieldChecker.check("projects", + Lists.newArrayList(pk1, + pk11.toSlot(), + pk2.toSlot(), + sumA1.toSlot(), + countA11.toSlot(), + sumA1A2.toSlot(), + v1.toSlot())))); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregateTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregateTest.java similarity index 99% rename from fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregateTest.java rename to fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregateTest.java index 32f7b324f9af47d..3808fd1842810f1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregateTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregateTest.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.nereids.rules.rewrite; +package org.apache.doris.nereids.rules.analysis; import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.Alias; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTest.java index 5b3cdd7dd19caf3..1010e7df27177fc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTest.java @@ -59,10 +59,10 @@ /** * all expr rewrite rule test case. */ -public class ExpressionRewriteTest extends ExpressionRewriteTestHelper { +class ExpressionRewriteTest extends ExpressionRewriteTestHelper { @Test - public void testNotRewrite() { + void testNotRewrite() { executor = new ExpressionRuleExecutor(ImmutableList.of(SimplifyNotExprRule.INSTANCE)); assertRewrite("not x", "not x"); @@ -87,7 +87,7 @@ public void testNotRewrite() { } @Test - public void testNormalizeExpressionRewrite() { + void testNormalizeExpressionRewrite() { executor = new ExpressionRuleExecutor(ImmutableList.of(NormalizeBinaryPredicatesRule.INSTANCE)); assertRewrite("1 = 1", "1 = 1"); @@ -99,7 +99,7 @@ public void testNormalizeExpressionRewrite() { } @Test - public void testDistinctPredicatesRewrite() { + void testDistinctPredicatesRewrite() { executor = new ExpressionRuleExecutor(ImmutableList.of(DistinctPredicatesRule.INSTANCE)); assertRewrite("a = 1", "a = 1"); @@ -111,7 +111,7 @@ public void testDistinctPredicatesRewrite() { } @Test - public void testExtractCommonFactorRewrite() { + void testExtractCommonFactorRewrite() { executor = new ExpressionRuleExecutor(ImmutableList.of(ExtractCommonFactorRule.INSTANCE)); assertRewrite("a", "a"); @@ -164,7 +164,7 @@ public void testExtractCommonFactorRewrite() { } @Test - public void testInPredicateToEqualToRule() { + void testInPredicateToEqualToRule() { executor = new ExpressionRuleExecutor(ImmutableList.of(InPredicateToEqualToRule.INSTANCE)); assertRewrite("a in (1)", "a = 1"); @@ -180,14 +180,14 @@ public void testInPredicateToEqualToRule() { } @Test - public void testInPredicateDedup() { + void testInPredicateDedup() { executor = new ExpressionRuleExecutor(ImmutableList.of(InPredicateDedup.INSTANCE)); assertRewrite("a in (1, 2, 1, 2)", "a in (1, 2)"); } @Test - public void testSimplifyCastRule() { + void testSimplifyCastRule() { executor = new ExpressionRuleExecutor(ImmutableList.of(SimplifyCastRule.INSTANCE)); // deduplicate @@ -219,7 +219,7 @@ public void testSimplifyCastRule() { } @Test - public void testSimplifyComparisonPredicateRule() { + void testSimplifyComparisonPredicateRule() { executor = new ExpressionRuleExecutor(ImmutableList.of(SimplifyCastRule.INSTANCE, SimplifyComparisonPredicate.INSTANCE)); Expression dtv2 = new DateTimeV2Literal(1, 1, 1, 1, 1, 1, 0); @@ -271,7 +271,7 @@ public void testSimplifyComparisonPredicateRule() { } @Test - public void testSimplifyDecimalV3Comparison() { + void testSimplifyDecimalV3Comparison() { executor = new ExpressionRuleExecutor(ImmutableList.of(SimplifyDecimalV3Comparison.INSTANCE)); // do rewrite diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java index 8b9f48010aae6eb..567073254a9376c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyArithmeticRuleTest.java @@ -36,7 +36,7 @@ public void testSimplifyArithmetic() { assertRewriteAfterTypeCoercion("IA", "IA"); assertRewriteAfterTypeCoercion("IA + 1", "IA + 1"); assertRewriteAfterTypeCoercion("IA + IB", "IA + IB"); - assertRewriteAfterTypeCoercion("1 * 3 / IA", "(3.0 / cast(IA as DOUBLE))"); + assertRewriteAfterTypeCoercion("1 * 3 / IA", "(3 / cast(IA as DOUBLE))"); assertRewriteAfterTypeCoercion("1 - IA", "1 - IA"); assertRewriteAfterTypeCoercion("1 + 1", "2"); assertRewriteAfterTypeCoercion("IA + 2 - 1", "IA + 1"); @@ -44,12 +44,12 @@ public void testSimplifyArithmetic() { assertRewriteAfterTypeCoercion("IA + 2 - ((1 - IB) - (3 + IC))", "IA + IB + IC + 4"); assertRewriteAfterTypeCoercion("IA * IB + 2 - IC * 2", "(IA * IB) - (IC * 2) + 2"); assertRewriteAfterTypeCoercion("IA * IB", "IA * IB"); - assertRewriteAfterTypeCoercion("IA * IB / 2 * 2", "cast((IA * IB) as DOUBLE) / 1.0"); - assertRewriteAfterTypeCoercion("IA * IB / (2 * 2)", "cast((IA * IB) as DOUBLE) / 4.0"); - assertRewriteAfterTypeCoercion("IA * IB / (2 * 2)", "cast((IA * IB) as DOUBLE) / 4.0"); - assertRewriteAfterTypeCoercion("IA * (IB / 2) * 2)", "cast(IA as DOUBLE) * cast(IB as DOUBLE) / 1.0"); - assertRewriteAfterTypeCoercion("IA * (IB / 2) * (IC + 1))", "cast(IA as DOUBLE) * cast(IB as DOUBLE) * cast((IC + 1) as DOUBLE) / 2.0"); - assertRewriteAfterTypeCoercion("IA * IB / 2 / IC * 2 * ID / 4", "(((cast((IA * IB) as DOUBLE) / cast(IC as DOUBLE)) * cast(ID as DOUBLE)) / 4.0)"); + assertRewriteAfterTypeCoercion("IA * IB / 2 * 2", "cast((IA * IB) as DOUBLE) / 1"); + assertRewriteAfterTypeCoercion("IA * IB / (2 * 2)", "cast((IA * IB) as DOUBLE) / 4"); + assertRewriteAfterTypeCoercion("IA * IB / (2 * 2)", "cast((IA * IB) as DOUBLE) / 4"); + assertRewriteAfterTypeCoercion("IA * (IB / 2) * 2)", "cast(IA as DOUBLE) * cast(IB as DOUBLE) / 1"); + assertRewriteAfterTypeCoercion("IA * (IB / 2) * (IC + 1))", "cast(IA as DOUBLE) * cast(IB as DOUBLE) * cast((IC + 1) as DOUBLE) / 2"); + assertRewriteAfterTypeCoercion("IA * IB / 2 / IC * 2 * ID / 4", "(((cast((IA * IB) as DOUBLE) / cast(IC as DOUBLE)) * cast(ID as DOUBLE)) / 4)"); } @Test @@ -86,8 +86,8 @@ public void testSimplifyArithmeticComparison() { assertRewriteAfterTypeCoercion("IA + 1 > IB", "cast(IA as BIGINT) > (cast(IB as BIGINT) - 1)"); assertRewriteAfterTypeCoercion("IA + 1 > IB * IC", "cast(IA as BIGINT) > ((IB * IC) - 1)"); assertRewriteAfterTypeCoercion("IA * ID > IB * IC", "IA * ID > IB * IC"); - assertRewriteAfterTypeCoercion("IA * ID / 2 > IB * IC", "cast((IA * ID) as DOUBLE) > cast((IB * IC) as DOUBLE) * 2.0"); - assertRewriteAfterTypeCoercion("IA * ID / -2 > IB * IC", "cast((IB * IC) as DOUBLE) * -2.0 > cast((IA * ID) as DOUBLE)"); + assertRewriteAfterTypeCoercion("IA * ID / 2 > IB * IC", "cast((IA * ID) as DOUBLE) > cast((IB * IC) as DOUBLE) * 2"); + assertRewriteAfterTypeCoercion("IA * ID / -2 > IB * IC", "cast((IB * IC) as DOUBLE) * -2 > cast((IA * ID) as DOUBLE)"); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java index 6f3bfaa7e533142..34c163091814667 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java @@ -20,6 +20,7 @@ import org.apache.doris.nereids.annotation.Developing; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.analysis.NormalizeAggregate; import org.apache.doris.nereids.rules.implementation.AggregateStrategies; import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.AggregateExpression; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java index 04e84ab8e89d860..5c43d7274d3fcc3 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java @@ -299,15 +299,16 @@ public void pruneAggregateOutput() { .matches( logicalProject( logicalSubQueryAlias( - logicalAggregate( logicalProject( - logicalOlapScan() - ).when(p -> getOutputQualifiedNames(p).equals( - ImmutableList.of("default_cluster:test.student.id") - )) - ).when(agg -> getOutputQualifiedNames(agg.getOutputs()).equals( - ImmutableList.of("default_cluster:test.student.id") - )) + logicalAggregate( + logicalProject( + logicalOlapScan() + ).when(p -> getOutputQualifiedNames(p).equals( + ImmutableList.of("default_cluster:test.student.id") + )) + ).when(agg -> getOutputQualifiedNames(agg.getOutputs()).equals( + ImmutableList.of("default_cluster:test.student.id") + ))) ) ) ); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpressionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpressionTest.java index e676caa37a8c984..476131e6b068b15 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpressionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpressionTest.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.rules.rewrite; import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.rules.analysis.NormalizeAggregate; import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownAliasThroughJoinTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownAliasThroughJoinTest.java index 5667f3f2c5ad3d9..5a98b07bcf041cc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownAliasThroughJoinTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownAliasThroughJoinTest.java @@ -18,6 +18,10 @@ package org.apache.doris.nereids.rules.rewrite; import org.apache.doris.common.Pair; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.MarkJoinSlotReference; +import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; @@ -30,6 +34,8 @@ import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Test; +import java.util.List; + class PushdownAliasThroughJoinTest implements MemoPatternMatchSupported { private static final LogicalOlapScan scan1 = PlanConstructor.newLogicalOlapScan(0, "t1", 0); private static final LogicalOlapScan scan2 = PlanConstructor.newLogicalOlapScan(1, "t2", 0); @@ -99,4 +105,21 @@ void testJustRightSide() { && project.getProjects().get(1).toSql().equals("2name")) ); } + + @Test + void testNoPushdownMarkJoin() { + List projects = + ImmutableList.of(new MarkJoinSlotReference(new ExprId(101), "markSlot1", false), + new Alias(new MarkJoinSlotReference(new ExprId(102), "markSlot2", false), + "markSlot2")); + LogicalPlan plan = new LogicalPlanBuilder(scan1) + .join(scan2, JoinType.INNER_JOIN, Pair.of(0, 0)).projectExprs(projects).build(); + + PlanChecker.from(MemoTestUtils.createConnectContext(), plan) + .applyTopDown(new PushdownAliasThroughJoin()) + .matches(logicalProject(logicalJoin(logicalOlapScan(), logicalOlapScan())) + .when(project -> project.getProjects().get(0).toSql().equals("markSlot1") + && project.getProjects().get(1).toSql() + .equals("markSlot2 AS `markSlot2`"))); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java index 29cc509d9549889..dfad75d5d8042ac 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java @@ -135,20 +135,22 @@ public void testAggNodeCase() { .applyTopDown(new FindHashConditionForJoin()) .applyTopDown(new PushdownExpressionsInHashCondition()) .matches( - logicalProject( - logicalJoin( - logicalProject( - logicalOlapScan() - ), - logicalProject( - logicalSubQueryAlias( - logicalAggregate( - logicalOlapScan() - ) + logicalProject( + logicalJoin( + logicalProject( + logicalOlapScan() + ), + logicalProject( + logicalSubQueryAlias( + logicalProject( + logicalAggregate( + logicalProject( + logicalOlapScan() + ))) + ) + ) ) - ) ) - ) ); } @@ -168,8 +170,12 @@ public void testSortNodeCase() { logicalProject( logicalSubQueryAlias( logicalSort( - logicalAggregate( - logicalOlapScan() + logicalProject( + logicalAggregate( + logicalProject( + logicalOlapScan() + ) + ) ) ) ) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java index a3bd46eb4f2f4ea..ed5a96933db1059 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java @@ -20,6 +20,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.nereids.rules.analysis.LogicalSubQueryAliasToLogicalProject; import org.apache.doris.nereids.rules.rewrite.MergeProjects; +import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughProject; import org.apache.doris.nereids.trees.plans.PreAggStatus; import org.apache.doris.nereids.util.MemoPatternMatchSupported; import org.apache.doris.nereids.util.PlanChecker; @@ -188,7 +189,8 @@ public void testWithFilterAndProject() { PlanChecker.from(connectContext) .analyze(sql) .applyBottomUp(new LogicalSubQueryAliasToLogicalProject()) - .applyTopDown(new MergeProjects()) + .applyTopDown(new PushdownFilterThroughProject()) + .applyBottomUp(new MergeProjects()) .applyTopDown(new SelectMaterializedIndexWithAggregate()) .applyTopDown(new SelectMaterializedIndexWithoutAggregate()) .matches(logicalOlapScan().when(scan -> { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java index c0a563085efa8b9..34cff095546936f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java @@ -70,7 +70,7 @@ protected void runBeforeAll() throws Exception { } @Test - public void testFromClauseUpdate() throws AnalysisException { + public void testFromClauseDelete() throws AnalysisException { String sql = "delete from t1 a using src join t2 on src.k1 = t2.k1 where t2.k1 = a.k1"; LogicalPlan parsed = new NereidsParser().parseSingle(sql); Assertions.assertTrue(parsed instanceof DeleteCommand); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/types/DataTypeTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/types/DataTypeTest.java index 9633db62496e73d..608130d6a21b13a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/types/DataTypeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/types/DataTypeTest.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.types.coercion.IntegralType; import org.apache.doris.nereids.types.coercion.NumericType; +import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -67,7 +68,6 @@ void testFromPrimitiveType() { @Test void testConvertFromString() { // boolean - Assertions.assertEquals(BooleanType.INSTANCE, DataType.convertFromString("bool")); Assertions.assertEquals(BooleanType.INSTANCE, DataType.convertFromString("boolean")); // tinyint Assertions.assertEquals(TinyIntType.INSTANCE, DataType.convertFromString("tinyint")); @@ -95,13 +95,11 @@ void testConvertFromString() { Assertions.assertEquals(StringType.INSTANCE, DataType.convertFromString("string")); // char Assertions.assertEquals(CharType.createCharType(10), DataType.convertFromString("char(10)")); - Assertions.assertEquals(CharType.createCharType(10), DataType.convertFromString("character(10)")); - + Assertions.assertEquals(CharType.SYSTEM_DEFAULT, DataType.convertFromString("character")); // varchar Assertions.assertEquals(VarcharType.createVarcharType(10), DataType.convertFromString("varchar(10)")); - // null - Assertions.assertEquals(NullType.INSTANCE, DataType.convertFromString("null")); - Assertions.assertEquals(NullType.INSTANCE, DataType.convertFromString("null_type")); + Assertions.assertEquals(VarcharType.SYSTEM_DEFAULT, DataType.convertFromString("varchar(*)")); + Assertions.assertEquals(VarcharType.SYSTEM_DEFAULT, DataType.convertFromString("varchar")); // date Assertions.assertEquals(DateType.INSTANCE, DataType.convertFromString("date")); // datev2 @@ -124,6 +122,11 @@ void testConvertFromString() { Assertions.assertEquals(JsonType.INSTANCE, DataType.convertFromString("json")); // array Assertions.assertEquals(ArrayType.of(IntegerType.INSTANCE), DataType.convertFromString("array")); + // map + Assertions.assertEquals(MapType.of(IntegerType.INSTANCE, IntegerType.INSTANCE), DataType.convertFromString("map")); + // struct + Assertions.assertEquals(new StructType(ImmutableList.of(new StructField("a", IntegerType.INSTANCE, true, ""))), DataType.convertFromString("struct")); + } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/PartitionCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/PartitionCacheTest.java index 4ccd783025c87e6..52ee9e009f58733 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/PartitionCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/PartitionCacheTest.java @@ -1104,6 +1104,7 @@ public void testSqlCacheKey() { Assert.assertEquals(cacheKey, "SELECT `eventdate` AS `eventdate`, count(`userid`) AS " + "`count(``userid``)` FROM `testCluster:testDb`.`appevent` WHERE `eventdate` >= '2020-01-12' AND " + "`eventdate` <= '2020-01-14' GROUP BY `eventdate`|"); + Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @Test @@ -1123,6 +1124,7 @@ public void testSqlCacheKeyWithChineseChar() { String cacheKey = sqlCache.getSqlWithViewStmt(); Types.PUniqueId sqlKey2 = CacheProxy.getMd5(cacheKey.replace("北京", "上海")); Assert.assertNotEquals(CacheProxy.getMd5(sqlCache.getSqlWithViewStmt()), sqlKey2); + Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @Test @@ -1141,6 +1143,7 @@ public void testSqlCacheKeyWithView() { Assert.assertEquals(cacheKey, "SELECT `testCluster:testDb`.`view1`.`eventdate` AS `eventdate`, `testCluster:testDb`.`view1`." + "`count(`userid`)` AS `count(``userid``)` FROM `testCluster:testDb`.`view1`|select eventdate, COUNT(userid) " + "FROM appevent WHERE eventdate>=\"2020-01-12\" and eventdate<=\"2020-01-14\" GROUP BY eventdate"); + Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @Test @@ -1159,6 +1162,7 @@ public void testSqlCacheKeyWithViewForNereids() { Assert.assertEquals(cacheKey, "SELECT * from testDb.view1" + "|select eventdate, COUNT(userid) FROM appevent " + "WHERE eventdate>=\"2020-01-12\" and eventdate<=\"2020-01-14\" GROUP BY eventdate"); + Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @Test @@ -1185,6 +1189,7 @@ public void testSqlCacheKeyWithSubSelectView() { + "`userid` FROM (SELECT `view2`.`eventdate` AS `eventdate`, `view2`.`userid` AS `userid` FROM " + "`testCluster:testDb`.`view2` view2 WHERE `view2`.`eventdate` >= '2020-01-12' AND `view2`.`eventdate` " + "<= '2020-01-14') origin|select eventdate, userid FROM appevent"); + Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @@ -1214,6 +1219,7 @@ public void testSqlCacheKeyWithSubSelectViewForNereids() { + " from testDb.view2 view2 \n" + " where view2.eventdate >=\"2020-01-12\" and view2.eventdate <= \"2020-01-14\"\n" + ") origin" + "|select eventdate, userid FROM appevent"); + Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @Test @@ -1292,6 +1298,7 @@ public void testSqlCacheKeyWithNestedView() { + "`testCluster:testDb`.`view4`.`count(`userid`)` AS `count(``userid``)` FROM `testCluster:testDb`.`view4`|select " + "eventdate, COUNT(userid) FROM view2 WHERE eventdate>=\"2020-01-12\" and " + "eventdate<=\"2020-01-14\" GROUP BY eventdate|select eventdate, userid FROM appevent"); + Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @Test @@ -1311,6 +1318,7 @@ public void testSqlCacheKeyWithNestedViewForNereids() { + "|select eventdate, COUNT(userid) FROM view2 " + "WHERE eventdate>=\"2020-01-12\" and eventdate<=\"2020-01-14\" GROUP BY eventdate" + "|select eventdate, userid FROM appevent"); + Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); } @Test @@ -1328,10 +1336,12 @@ public void testCacheLocalViewMultiOperand() { ); ArrayList selectedPartitionIds = Lists.newArrayList(20200112L, 20200113L, 20200114L, 20200115L); - List scanNodes = Lists.newArrayList(createProfileScanNode(selectedPartitionIds)); + ScanNode scanNode = createProfileScanNode(selectedPartitionIds); + List scanNodes = Lists.newArrayList(scanNode, scanNode, scanNode); CacheAnalyzer ca = new CacheAnalyzer(context, parseStmt, scanNodes); ca.checkCacheMode(0); Assert.assertEquals(ca.getCacheMode(), CacheMode.Sql); + Assert.assertEquals(selectedPartitionIds.size() * 3, ((SqlCache) ca.getCache()).getSumOfPartitionNum()); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java index e85bbf2c2820663..9624c20149828dd 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisJobTest.java @@ -135,6 +135,7 @@ public List executeInternalQuery() { .setAnalysisMethod(AnalysisMethod.FULL) .setAnalysisType(AnalysisType.FUNDAMENTALS) .setColToPartitions(colToPartitions) + .setState(AnalysisState.RUNNING) .build(); new OlapAnalysisTask(analysisJobInfo).doExecute(); new Expectations() { diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java index 4f7b1a9c5d24f07..16ef1705d88aed1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisManagerTest.java @@ -24,6 +24,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.statistics.AnalysisInfo.AnalysisType; import org.apache.doris.statistics.AnalysisInfo.JobType; +import org.apache.doris.statistics.AnalysisInfo.ScheduleType; import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.annotations.VisibleForTesting; @@ -182,7 +183,9 @@ public void updateTableStats(AnalysisInfo jobInfo) { // test build async job @Test public void testBuildAndAssignJob2(@Injectable OlapAnalysisTask analysisTask) throws Exception { - AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setColToPartitions(new HashMap<>()).build(); + AnalysisInfo analysisInfo = new AnalysisInfoBuilder().setColToPartitions(new HashMap<>()) + .setScheduleType(ScheduleType.PERIOD) + .build(); new MockUp() { @Mock @@ -240,9 +243,9 @@ public void logCreateAnalysisJob(AnalysisInfo analysisJob) { }, new AnalyzeProperties(new HashMap() { { put(AnalyzeProperties.PROPERTY_SYNC, "false"); + put(AnalyzeProperties.PROPERTY_PERIOD_SECONDS, "100"); } })); - AnalysisManager analysisManager = new AnalysisManager(); analysisInfo.colToPartitions.put("c1", new HashSet() { { diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java index 6136b8efd442e10..196ac8ad9a056f3 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/AnalysisTaskExecutorTest.java @@ -113,6 +113,7 @@ public void syncLoadColStats(long tableId, long idxId, String colName) { .setAnalysisMode(AnalysisMode.FULL) .setAnalysisMethod(AnalysisMethod.FULL) .setAnalysisType(AnalysisType.FUNDAMENTALS) + .setState(AnalysisState.RUNNING) .setColToPartitions(colToPartitions) .build(); OlapAnalysisTask task = new OlapAnalysisTask(analysisInfo); diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoAnalyzerTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoAnalyzerTest.java index 3368a5a6692b70d..fff649a4473eb74 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoAnalyzerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoAnalyzerTest.java @@ -39,7 +39,6 @@ import mockit.Injectable; import mockit.Mock; import mockit.MockUp; -import mockit.Mocked; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -140,14 +139,25 @@ public List getBaseSchema() { } @Test - public void testGetReAnalyzeRequiredPart0(@Mocked TableIf tableIf) { + public void testGetReAnalyzeRequiredPart0() { - new Expectations() { - { - tableIf.getRowCount(); - result = 100; + TableIf tableIf = new OlapTable(); + + new MockUp() { + @Mock + protected Set findReAnalyzeNeededPartitions(TableStats tableStats) { + Set partitionNames = new HashSet<>(); + partitionNames.add("p1"); + partitionNames.add("p2"); + return partitionNames; + } + + @Mock + public long getRowCount() { + return 100; } }; + new MockUp() { @Mock public TableIf findTable(String catalogName, String dbName, String tblName) { @@ -176,14 +186,6 @@ public TableStats findTableStatsStatus(long tblId) { }; new MockUp() { - @Mock - protected Set findReAnalyzeNeededPartitions(TableIf table, TableStats tableStats) { - Set partitionNames = new HashSet<>(); - partitionNames.add("p1"); - partitionNames.add("p2"); - return partitionNames; - } - @Mock public AnalysisInfo getAnalysisJobInfo(AnalysisInfo jobInfo, TableIf table, Set needRunPartitions) { diff --git a/fe/pom.xml b/fe/pom.xml index b1055b29630a6f8..b4d7841967230cf 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -257,7 +257,7 @@ under the License. 1.1.0.Final 0.2.3 3.4.0 - 4.0.0 + 6.4.5 3.8.9.Final 1.3.2 1.2.0 @@ -273,6 +273,7 @@ under the License. 1.1.0 + 3.0.0rc1 0.43.3-public 9.0.0 1.11.1 diff --git a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java index d86df86fe0b9526..d25947e33bf2701 100644 --- a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java +++ b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java @@ -1204,27 +1204,32 @@ public ByteBuffer pread(TBrokerFD fd, long offset, long length) { // Avoid using the ByteBuffer based read for Hadoop because some FSDataInputStream // implementations are not ByteBufferReadable, // See https://issues.apache.org/jira/browse/HADOOP-14603 - byte[] buf; - if (length > readBufferSize) { - buf = new byte[readBufferSize]; - } else { - buf = new byte[(int) length]; - } - try { - int readLength = readBytesFully(fsDataInputStream, buf); - if (readLength < 0) { - throw new BrokerException(TBrokerOperationStatusCode.END_OF_FILE, - "end of file reached"); - } - if (logger.isDebugEnabled()) { - logger.debug("read buffer from input stream, buffer size:" + buf.length + ", read length:" + readLength); + int hasRead = 0; + byte[] buf = new byte[(int)length]; + while (hasRead < length) { + int bufSize = Math.min((int) length - hasRead, readBufferSize); + try { + int readLength = fsDataInputStream.read(buf, hasRead, bufSize); + if (readLength < 0) { + throw new BrokerException(TBrokerOperationStatusCode.END_OF_FILE, + "end of file reached"); + } + if (logger.isDebugEnabled()) { + logger.debug("read buffer from input stream, buffer size:" + buf.length + ", read length:" + + readLength); + } + hasRead += readLength; + } catch (IOException e) { + logger.error("errors while read data from stream", e); + throw new BrokerException(TBrokerOperationStatusCode.TARGET_STORAGE_SERVICE_ERROR, + e, "errors while read data from stream"); } - return ByteBuffer.wrap(buf, 0, readLength); - } catch (IOException e) { - logger.error("errors while read data from stream", e); + } + if (hasRead != length) { throw new BrokerException(TBrokerOperationStatusCode.TARGET_STORAGE_SERVICE_ERROR, - e, "errors while write data to output stream"); + String.format("errors while read data from stream: hasRead(%d) != length(%d)", hasRead, length)); } + return ByteBuffer.wrap(buf, 0, hasRead); } } @@ -1325,19 +1330,6 @@ private static TBrokerFD parseUUIDToFD(UUID uuid) { return new TBrokerFD(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); } - private int readBytesFully(FSDataInputStream is, byte[] dest) throws IOException { - int readLength = 0; - while (readLength < dest.length) { - int availableReadLength = dest.length - readLength; - int n = is.read(dest, readLength, availableReadLength); - if (n <= 0) { - break; - } - readLength += n; - } - return readLength; - } - /** * In view of the different expiration mechanisms of different authentication modes, * there are two ways to determine whether BrokerFileSystem has expired: diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index bdfc0d823a51356..5a462987d2e5311 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -301,6 +301,7 @@ message PCacheParam { required int64 partition_key = 1; optional int64 last_version = 2; optional int64 last_version_time = 3; + optional int64 partition_num = 4; }; message PCacheValue { diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index fe3ac7915c97043..6a21a7773446c31 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -149,12 +149,21 @@ message DeletePredicatePB { required int32 version = 1; // This field is useless, but could not removed, not depend on it repeated string sub_predicates = 2; repeated InPredicatePB in_predicates = 3; + repeated DeleteSubPredicatePB sub_predicates_v2 = 4; +} + +message DeleteSubPredicatePB { + optional int32 column_unique_id = 1; + optional string column_name = 2; + optional string op = 3; + optional string cond_value = 4; } message InPredicatePB { optional string column_name = 1; optional bool is_not_in = 2; repeated string values = 3; + optional int32 column_unique_id = 4; } enum AlterTabletState { diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index d707b95f1a65bd6..c0bec45f35146ee 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -103,12 +103,12 @@ [['array'], 'ARRAY', ['DATE', '...'], 'ALWAYS_NOT_NULLABLE'], [['array'], 'ARRAY', ['DATETIMEV2', '...'], 'ALWAYS_NOT_NULLABLE'], [['array'], 'ARRAY', ['DATEV2', '...'], 'ALWAYS_NOT_NULLABLE'], - [['array'], 'ARRAY', ['FLOAT', '...'], 'ALWAYS_NOT_NULLABLE'], - [['array'], 'ARRAY', ['DOUBLE', '...'], 'ALWAYS_NOT_NULLABLE'], [['array'], 'ARRAY', ['DECIMALV2', '...'], 'ALWAYS_NOT_NULLABLE'], [['array'], 'ARRAY', ['DECIMAL32', '...'], 'ALWAYS_NOT_NULLABLE'], [['array'], 'ARRAY', ['DECIMAL64', '...'], 'ALWAYS_NOT_NULLABLE'], [['array'], 'ARRAY', ['DECIMAL128', '...'], 'ALWAYS_NOT_NULLABLE'], + [['array'], 'ARRAY', ['FLOAT', '...'], 'ALWAYS_NOT_NULLABLE'], + [['array'], 'ARRAY', ['DOUBLE', '...'], 'ALWAYS_NOT_NULLABLE'], [['array'], 'ARRAY', ['VARCHAR', '...'], 'ALWAYS_NOT_NULLABLE'], [['array'], 'ARRAY', ['STRING', '...'], 'ALWAYS_NOT_NULLABLE'], @@ -890,9 +890,11 @@ [['to_monday'], 'DATE', ['DATE'], 'ALWAYS_NULLABLE'], [['to_days'], 'INT', ['DATE'], 'ALWAYS_NULLABLE'], - [['date_trunc'], 'DATETIME', ['DATETIME', 'VARCHAR'], 'ALWAYS_NULLABLE'], + [['date_floor'], 'DATETIMEV2', ['DATETIMEV2', 'INT'], 'ALWAYS_NULLABLE'], + [['date_ceil'], 'DATETIMEV2', ['DATETIMEV2', 'INT'], 'ALWAYS_NULLABLE'], [['date_trunc'], 'DATETIMEV2', ['DATETIMEV2', 'VARCHAR'], 'ALWAYS_NULLABLE'], + [['date_trunc'], 'DATETIME', ['DATETIME', 'VARCHAR'], 'ALWAYS_NULLABLE'], [['year'], 'SMALLINT', ['DATETIME'], 'ALWAYS_NULLABLE'], [['month'], 'TINYINT', ['DATETIME'], 'ALWAYS_NULLABLE'], diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index cb64ddb70dd4f95..302057087685449 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -117,7 +117,9 @@ enum TSchemaTableType { SCH_ROWSETS, SCH_BACKENDS, SCH_COLUMN_STATISTICS, - SCH_PARAMETERS; + SCH_PARAMETERS, + SCH_METADATA_NAME_IDS, + SCH_PROFILING; } enum THdfsCompression { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index c1023331a2b91eb..7199484ed17089d 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -308,12 +308,15 @@ struct TGetDbsParams { 3: optional string user_ip // deprecated 4: optional Types.TUserIdentity current_user_ident // to replace the user and user ip 5: optional string catalog + 6: optional bool get_null_catalog //if catalog is empty , get dbName ="NULL" and dbId = -1. } -// getDbNames returns a list of database names and catalog names +// getDbNames returns a list of database names , database ids and catalog names ,catalog ids struct TGetDbsResult { 1: optional list dbs 2: optional list catalogs + 3: optional list db_ids + 4: optional list catalog_ids } // Arguments to getTableNames, which returns a list of tables that match an @@ -351,6 +354,15 @@ struct TListTableStatusResult { 1: required list tables } +struct TTableMetadataNameIds { + 1: optional string name + 2: optional i64 id +} + +struct TListTableMetadataNameIdsResult { + 1: optional list tables +} + // getTableNames returns a list of unqualified table names struct TGetTablesResult { 1: list tables @@ -631,11 +643,6 @@ struct TStreamLoadMultiTablePutResult { 3: optional list pipeline_params } -// StreamLoadWith request status -struct TStreamLoadWithLoadStatusRequest { - 1: optional Types.TUniqueId loadId -} - struct TStreamLoadWithLoadStatusResult { 1: optional Status.TStatus status 2: optional i64 txn_id @@ -803,6 +810,7 @@ struct TFrontendPingFrontendResult { 6: required string version 7: optional i64 lastStartupTime 8: optional list diskInfos + 9: optional i64 processUUID } struct TPropertyVal { @@ -1135,6 +1143,7 @@ service FrontendService { TMasterOpResult forward(1: TMasterOpRequest params) TListTableStatusResult listTableStatus(1: TGetTablesParams params) + TListTableMetadataNameIdsResult listTableMetadataNameIds(1: TGetTablesParams params) TListPrivilegesResult listTablePrivilegeStatus(1: TGetTablesParams params) TListPrivilegesResult listSchemaPrivilegeStatus(1: TGetTablesParams params) TListPrivilegesResult listUserPrivilegeStatus(1: TGetTablesParams params) @@ -1157,7 +1166,6 @@ service FrontendService { TWaitingTxnStatusResult waitingTxnStatus(1: TWaitingTxnStatusRequest request) TStreamLoadPutResult streamLoadPut(1: TStreamLoadPutRequest request) - TStreamLoadWithLoadStatusResult streamLoadWithLoadStatus(1: TStreamLoadWithLoadStatusRequest request) TStreamLoadMultiTablePutResult streamLoadMultiTablePut(1: TStreamLoadPutRequest request) diff --git a/gensrc/thrift/HeartbeatService.thrift b/gensrc/thrift/HeartbeatService.thrift index 38ea17e907ee0b5..b9b18a2b85f0017 100644 --- a/gensrc/thrift/HeartbeatService.thrift +++ b/gensrc/thrift/HeartbeatService.thrift @@ -24,6 +24,11 @@ include "Types.thrift" const i64 IS_SET_DEFAULT_ROWSET_TO_BETA_BIT = 0x01; +struct TFrontendInfo { + 1: optional Types.TNetworkAddress coordinator_address + 2: optional i64 process_uuid +} + struct TMasterInfo { 1: required Types.TNetworkAddress network_address 2: required Types.TClusterId cluster_id @@ -33,6 +38,7 @@ struct TMasterInfo { 6: optional Types.TPort http_port 7: optional i64 heartbeat_flags 8: optional i64 backend_id + 9: optional list frontend_infos } struct TBackendInfo { diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index ba204f474ed60ff..2f20057840018ed 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -235,6 +235,11 @@ struct TQueryOptions { 79: optional bool enable_pipeline_x_engine = false; 80: optional bool enable_memtable_on_sink_node = false; + + 81: optional bool enable_delete_sub_predicate_v2 = false; + + // A tag used to distinguish fe start epoch. + 82: optional i64 fe_process_uuid = 0; } diff --git a/regression-test/data/compaction/test_full_compaction.out b/regression-test/data/compaction/test_full_compaction.out index b5ed2dffc14e202..ead5be01b4e6b0c 100644 --- a/regression-test/data/compaction/test_full_compaction.out +++ b/regression-test/data/compaction/test_full_compaction.out @@ -32,8 +32,8 @@ 2 2 2 20 2 200 -3 300 3 100 +3 300 -- !select_final -- 1 100 diff --git a/regression-test/data/datatype_p0/decimalv3/test_decimalv3_overflow.out b/regression-test/data/datatype_p0/decimalv3/test_decimalv3_overflow.out new file mode 100644 index 000000000000000..9ffc65665a182de --- /dev/null +++ b/regression-test/data/datatype_p0/decimalv3/test_decimalv3_overflow.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +9999999999999999999999999999.9999999999 + +-- !sql -- +\N + diff --git a/regression-test/data/export_p0/test_export_basic.out b/regression-test/data/export_p0/test_export_basic.out index 643a79881e8740c..5afc94bb23923c1 100644 --- a/regression-test/data/export_p0/test_export_basic.out +++ b/regression-test/data/export_p0/test_export_basic.out @@ -377,6 +377,37 @@ 69 ftw-69 87 -- !select_load3 -- +70 ftw-70 88 +71 ftw-71 89 +72 ftw-72 90 +73 ftw-73 91 +74 ftw-74 92 +75 ftw-75 93 +76 ftw-76 94 +77 ftw-77 95 +78 ftw-78 96 +79 ftw-79 97 +80 ftw-80 98 +81 ftw-81 99 +82 ftw-82 100 +83 ftw-83 101 +84 ftw-84 102 +85 ftw-85 103 +86 ftw-86 104 +87 ftw-87 105 +88 ftw-88 106 +89 ftw-89 107 +90 ftw-90 108 +91 ftw-91 109 +92 ftw-92 110 +93 ftw-93 111 +94 ftw-94 112 +95 ftw-95 113 +96 ftw-96 114 +97 ftw-97 115 +98 ftw-98 116 +99 ftw-99 117 +100 ftw-100 118 101 ftw-101 119 102 ftw-102 120 103 ftw-103 121 diff --git a/regression-test/data/export_p0/test_export_csv.out b/regression-test/data/export_p0/test_export_csv.out index a712d0931cec0e8..ca4469c87fd162b 100644 --- a/regression-test/data/export_p0/test_export_csv.out +++ b/regression-test/data/export_p0/test_export_csv.out @@ -214,6 +214,96 @@ 8 2017-10-01 2017-10-01T00:00 Beijing 8 8 true 8 8 8 8.8 8.8 char8 8 9 2017-10-01 2017-10-01T00:00 Beijing 9 9 true 9 9 9 9.9 9.9 char9 9 10 2017-10-01 2017-10-01T00:00 Beijing 10 10 true 10 10 10 10.1 10.1 char10 10 +11 2017-10-01 2017-10-01T00:00 Beijing 11 11 true 11 11 11 11.11 11.11 char11 11 +12 2017-10-01 2017-10-01T00:00 Beijing 12 12 true 12 12 12 12.12 12.12 char12 12 +13 2017-10-01 2017-10-01T00:00 Beijing 13 13 true 13 13 13 13.13 13.13 char13 13 +14 2017-10-01 2017-10-01T00:00 Beijing 14 14 true 14 14 14 14.14 14.14 char14 14 +15 2017-10-01 2017-10-01T00:00 Beijing 15 15 true 15 15 15 15.15 15.15 char15 15 +16 2017-10-01 2017-10-01T00:00 Beijing 16 16 true 16 16 16 16.16 16.16 char16 16 +17 2017-10-01 2017-10-01T00:00 Beijing 17 17 true 17 17 17 17.17 17.17 char17 17 +18 2017-10-01 2017-10-01T00:00 Beijing 18 18 true 18 18 18 18.18 18.18 char18 18 +19 2017-10-01 2017-10-01T00:00 Beijing 19 19 true 19 19 19 19.19 19.19 char19 19 +20 2017-10-01 2017-10-01T00:00 Beijing 20 20 true 20 20 20 20.2 20.2 char20 20 +21 2017-10-01 2017-10-01T00:00 Beijing 21 21 true 21 21 21 21.21 21.21 char21 21 +22 2017-10-01 2017-10-01T00:00 Beijing 22 22 true 22 22 22 22.22 22.22 char22 22 +23 2017-10-01 2017-10-01T00:00 Beijing 23 23 true 23 23 23 23.23 23.23 char23 23 +24 2017-10-01 2017-10-01T00:00 Beijing 24 24 true 24 24 24 24.24 24.24 char24 24 +25 2017-10-01 2017-10-01T00:00 Beijing 25 25 true 25 25 25 25.25 25.25 char25 25 +26 2017-10-01 2017-10-01T00:00 Beijing 26 26 true 26 26 26 26.26 26.26 char26 26 +27 2017-10-01 2017-10-01T00:00 Beijing 27 27 true 27 27 27 27.27 27.27 char27 27 +28 2017-10-01 2017-10-01T00:00 Beijing 28 28 true 28 28 28 28.28 28.28 char28 28 +29 2017-10-01 2017-10-01T00:00 Beijing 29 29 true 29 29 29 29.29 29.29 char29 29 +30 2017-10-01 2017-10-01T00:00 Beijing 30 30 true 30 30 30 30.3 30.3 char30 30 +31 2017-10-01 2017-10-01T00:00 Beijing 31 31 true 31 31 31 31.31 31.31 char31 31 +32 2017-10-01 2017-10-01T00:00 Beijing 32 32 true 32 32 32 32.32 32.32 char32 32 +33 2017-10-01 2017-10-01T00:00 Beijing 33 33 true 33 33 33 33.33 33.33 char33 33 +34 2017-10-01 2017-10-01T00:00 Beijing 34 34 true 34 34 34 34.34 34.34 char34 34 +35 2017-10-01 2017-10-01T00:00 Beijing 35 35 true 35 35 35 35.35 35.35 char35 35 +36 2017-10-01 2017-10-01T00:00 Beijing 36 36 true 36 36 36 36.36 36.36 char36 36 +37 2017-10-01 2017-10-01T00:00 Beijing 37 37 true 37 37 37 37.37 37.37 char37 37 +38 2017-10-01 2017-10-01T00:00 Beijing 38 38 true 38 38 38 38.38 38.38 char38 38 +39 2017-10-01 2017-10-01T00:00 Beijing 39 39 true 39 39 39 39.39 39.39 char39 39 +40 2017-10-01 2017-10-01T00:00 Beijing 40 40 true 40 40 40 40.4 40.4 char40 40 +41 2017-10-01 2017-10-01T00:00 Beijing 41 41 true 41 41 41 41.41 41.41 char41 41 +42 2017-10-01 2017-10-01T00:00 Beijing 42 42 true 42 42 42 42.42 42.42 char42 42 +43 2017-10-01 2017-10-01T00:00 Beijing 43 43 true 43 43 43 43.43 43.43 char43 43 +44 2017-10-01 2017-10-01T00:00 Beijing 44 44 true 44 44 44 44.44 44.44 char44 44 +45 2017-10-01 2017-10-01T00:00 Beijing 45 45 true 45 45 45 45.45 45.45 char45 45 +46 2017-10-01 2017-10-01T00:00 Beijing 46 46 true 46 46 46 46.46 46.46 char46 46 +47 2017-10-01 2017-10-01T00:00 Beijing 47 47 true 47 47 47 47.47 47.47 char47 47 +48 2017-10-01 2017-10-01T00:00 Beijing 48 48 true 48 48 48 48.48 48.48 char48 48 +49 2017-10-01 2017-10-01T00:00 Beijing 49 49 true 49 49 49 49.49 49.49 char49 49 +50 2017-10-01 2017-10-01T00:00 Beijing 50 50 true 50 50 50 50.5 50.5 char50 50 +51 2017-10-01 2017-10-01T00:00 Beijing 51 51 true 51 51 51 51.51 51.51 char51 51 +52 2017-10-01 2017-10-01T00:00 Beijing 52 52 true 52 52 52 52.52 52.52 char52 52 +53 2017-10-01 2017-10-01T00:00 Beijing 53 53 true 53 53 53 53.53 53.53 char53 53 +54 2017-10-01 2017-10-01T00:00 Beijing 54 54 true 54 54 54 54.54 54.54 char54 54 +55 2017-10-01 2017-10-01T00:00 Beijing 55 55 true 55 55 55 55.55 55.55 char55 55 +56 2017-10-01 2017-10-01T00:00 Beijing 56 56 true 56 56 56 56.56 56.56 char56 56 +57 2017-10-01 2017-10-01T00:00 Beijing 57 57 true 57 57 57 57.57 57.57 char57 57 +58 2017-10-01 2017-10-01T00:00 Beijing 58 58 true 58 58 58 58.58 58.58 char58 58 +59 2017-10-01 2017-10-01T00:00 Beijing 59 59 true 59 59 59 59.59 59.59 char59 59 +60 2017-10-01 2017-10-01T00:00 Beijing 60 60 true 60 60 60 60.6 60.6 char60 60 +61 2017-10-01 2017-10-01T00:00 Beijing 61 61 true 61 61 61 61.61 61.61 char61 61 +62 2017-10-01 2017-10-01T00:00 Beijing 62 62 true 62 62 62 62.62 62.62 char62 62 +63 2017-10-01 2017-10-01T00:00 Beijing 63 63 true 63 63 63 63.63 63.63 char63 63 +64 2017-10-01 2017-10-01T00:00 Beijing 64 64 true 64 64 64 64.64 64.64 char64 64 +65 2017-10-01 2017-10-01T00:00 Beijing 65 65 true 65 65 65 65.65 65.65 char65 65 +66 2017-10-01 2017-10-01T00:00 Beijing 66 66 true 66 66 66 66.66 66.66 char66 66 +67 2017-10-01 2017-10-01T00:00 Beijing 67 67 true 67 67 67 67.67 67.67 char67 67 +68 2017-10-01 2017-10-01T00:00 Beijing 68 68 true 68 68 68 68.68 68.68 char68 68 +69 2017-10-01 2017-10-01T00:00 Beijing 69 69 true 69 69 69 69.69 69.69 char69 69 +70 2017-10-01 2017-10-01T00:00 Beijing 70 70 true 70 70 70 70.7 70.7 char70 70 +71 2017-10-01 2017-10-01T00:00 Beijing 71 71 true 71 71 71 71.71 71.71 char71 71 +72 2017-10-01 2017-10-01T00:00 Beijing 72 72 true 72 72 72 72.72 72.72 char72 72 +73 2017-10-01 2017-10-01T00:00 Beijing 73 73 true 73 73 73 73.73 73.73 char73 73 +74 2017-10-01 2017-10-01T00:00 Beijing 74 74 true 74 74 74 74.74 74.74 char74 74 +75 2017-10-01 2017-10-01T00:00 Beijing 75 75 true 75 75 75 75.75 75.75 char75 75 +76 2017-10-01 2017-10-01T00:00 Beijing 76 76 true 76 76 76 76.76 76.76 char76 76 +77 2017-10-01 2017-10-01T00:00 Beijing 77 77 true 77 77 77 77.77 77.77 char77 77 +78 2017-10-01 2017-10-01T00:00 Beijing 78 78 true 78 78 78 78.78 78.78 char78 78 +79 2017-10-01 2017-10-01T00:00 Beijing 79 79 true 79 79 79 79.79 79.79 char79 79 +80 2017-10-01 2017-10-01T00:00 Beijing 80 80 true 80 80 80 80.8 80.8 char80 80 +81 2017-10-01 2017-10-01T00:00 Beijing 81 81 true 81 81 81 81.81 81.81 char81 81 +82 2017-10-01 2017-10-01T00:00 Beijing 82 82 true 82 82 82 82.82 82.82 char82 82 +83 2017-10-01 2017-10-01T00:00 Beijing 83 83 true 83 83 83 83.83 83.83 char83 83 +84 2017-10-01 2017-10-01T00:00 Beijing 84 84 true 84 84 84 84.84 84.84 char84 84 +85 2017-10-01 2017-10-01T00:00 Beijing 85 85 true 85 85 85 85.85 85.85 char85 85 +86 2017-10-01 2017-10-01T00:00 Beijing 86 86 true 86 86 86 86.86 86.86 char86 86 +87 2017-10-01 2017-10-01T00:00 Beijing 87 87 true 87 87 87 87.87 87.87 char87 87 +88 2017-10-01 2017-10-01T00:00 Beijing 88 88 true 88 88 88 88.88 88.88 char88 88 +89 2017-10-01 2017-10-01T00:00 Beijing 89 89 true 89 89 89 89.89 89.89 char89 89 +90 2017-10-01 2017-10-01T00:00 Beijing 90 90 true 90 90 90 90.9 90.9 char90 90 +91 2017-10-01 2017-10-01T00:00 Beijing 91 91 true 91 91 91 91.91 91.91 char91 91 +92 2017-10-01 2017-10-01T00:00 Beijing 92 92 true 92 92 92 92.92 92.92 char92 92 +93 2017-10-01 2017-10-01T00:00 Beijing 93 93 true 93 93 93 93.93 93.93 char93 93 +94 2017-10-01 2017-10-01T00:00 Beijing 94 94 true 94 94 94 94.94 94.94 char94 94 +95 2017-10-01 2017-10-01T00:00 Beijing 95 95 true 95 95 95 95.95 95.95 char95 95 +96 2017-10-01 2017-10-01T00:00 Beijing 96 96 true 96 96 96 96.96 96.96 char96 96 +97 2017-10-01 2017-10-01T00:00 Beijing 97 97 true 97 97 97 97.97 97.97 char97 97 +98 2017-10-01 2017-10-01T00:00 Beijing 98 98 true 98 98 98 98.98 98.98 char98 98 +99 2017-10-01 2017-10-01T00:00 Beijing 99 99 true 99 99 99 99.99 99.99 char99 99 +100 2017-10-01 2017-10-01T00:00 \N \N \N \N \N \N \N \N \N \N \N -- !select_load3 -- 1 2017-10-01 2017-10-01T00:00 Beijing 1 1 true 1 1 1 1.1 1.1 char1 1 @@ -226,6 +316,96 @@ 8 2017-10-01 2017-10-01T00:00 Beijing 8 8 true 8 8 8 8.8 8.8 char8 8 9 2017-10-01 2017-10-01T00:00 Beijing 9 9 true 9 9 9 9.9 9.9 char9 9 10 2017-10-01 2017-10-01T00:00 Beijing 10 10 true 10 10 10 10.1 10.1 char10 10 +11 2017-10-01 2017-10-01T00:00 Beijing 11 11 true 11 11 11 11.11 11.11 char11 11 +12 2017-10-01 2017-10-01T00:00 Beijing 12 12 true 12 12 12 12.12 12.12 char12 12 +13 2017-10-01 2017-10-01T00:00 Beijing 13 13 true 13 13 13 13.13 13.13 char13 13 +14 2017-10-01 2017-10-01T00:00 Beijing 14 14 true 14 14 14 14.14 14.14 char14 14 +15 2017-10-01 2017-10-01T00:00 Beijing 15 15 true 15 15 15 15.15 15.15 char15 15 +16 2017-10-01 2017-10-01T00:00 Beijing 16 16 true 16 16 16 16.16 16.16 char16 16 +17 2017-10-01 2017-10-01T00:00 Beijing 17 17 true 17 17 17 17.17 17.17 char17 17 +18 2017-10-01 2017-10-01T00:00 Beijing 18 18 true 18 18 18 18.18 18.18 char18 18 +19 2017-10-01 2017-10-01T00:00 Beijing 19 19 true 19 19 19 19.19 19.19 char19 19 +20 2017-10-01 2017-10-01T00:00 Beijing 20 20 true 20 20 20 20.2 20.2 char20 20 +21 2017-10-01 2017-10-01T00:00 Beijing 21 21 true 21 21 21 21.21 21.21 char21 21 +22 2017-10-01 2017-10-01T00:00 Beijing 22 22 true 22 22 22 22.22 22.22 char22 22 +23 2017-10-01 2017-10-01T00:00 Beijing 23 23 true 23 23 23 23.23 23.23 char23 23 +24 2017-10-01 2017-10-01T00:00 Beijing 24 24 true 24 24 24 24.24 24.24 char24 24 +25 2017-10-01 2017-10-01T00:00 Beijing 25 25 true 25 25 25 25.25 25.25 char25 25 +26 2017-10-01 2017-10-01T00:00 Beijing 26 26 true 26 26 26 26.26 26.26 char26 26 +27 2017-10-01 2017-10-01T00:00 Beijing 27 27 true 27 27 27 27.27 27.27 char27 27 +28 2017-10-01 2017-10-01T00:00 Beijing 28 28 true 28 28 28 28.28 28.28 char28 28 +29 2017-10-01 2017-10-01T00:00 Beijing 29 29 true 29 29 29 29.29 29.29 char29 29 +30 2017-10-01 2017-10-01T00:00 Beijing 30 30 true 30 30 30 30.3 30.3 char30 30 +31 2017-10-01 2017-10-01T00:00 Beijing 31 31 true 31 31 31 31.31 31.31 char31 31 +32 2017-10-01 2017-10-01T00:00 Beijing 32 32 true 32 32 32 32.32 32.32 char32 32 +33 2017-10-01 2017-10-01T00:00 Beijing 33 33 true 33 33 33 33.33 33.33 char33 33 +34 2017-10-01 2017-10-01T00:00 Beijing 34 34 true 34 34 34 34.34 34.34 char34 34 +35 2017-10-01 2017-10-01T00:00 Beijing 35 35 true 35 35 35 35.35 35.35 char35 35 +36 2017-10-01 2017-10-01T00:00 Beijing 36 36 true 36 36 36 36.36 36.36 char36 36 +37 2017-10-01 2017-10-01T00:00 Beijing 37 37 true 37 37 37 37.37 37.37 char37 37 +38 2017-10-01 2017-10-01T00:00 Beijing 38 38 true 38 38 38 38.38 38.38 char38 38 +39 2017-10-01 2017-10-01T00:00 Beijing 39 39 true 39 39 39 39.39 39.39 char39 39 +40 2017-10-01 2017-10-01T00:00 Beijing 40 40 true 40 40 40 40.4 40.4 char40 40 +41 2017-10-01 2017-10-01T00:00 Beijing 41 41 true 41 41 41 41.41 41.41 char41 41 +42 2017-10-01 2017-10-01T00:00 Beijing 42 42 true 42 42 42 42.42 42.42 char42 42 +43 2017-10-01 2017-10-01T00:00 Beijing 43 43 true 43 43 43 43.43 43.43 char43 43 +44 2017-10-01 2017-10-01T00:00 Beijing 44 44 true 44 44 44 44.44 44.44 char44 44 +45 2017-10-01 2017-10-01T00:00 Beijing 45 45 true 45 45 45 45.45 45.45 char45 45 +46 2017-10-01 2017-10-01T00:00 Beijing 46 46 true 46 46 46 46.46 46.46 char46 46 +47 2017-10-01 2017-10-01T00:00 Beijing 47 47 true 47 47 47 47.47 47.47 char47 47 +48 2017-10-01 2017-10-01T00:00 Beijing 48 48 true 48 48 48 48.48 48.48 char48 48 +49 2017-10-01 2017-10-01T00:00 Beijing 49 49 true 49 49 49 49.49 49.49 char49 49 +50 2017-10-01 2017-10-01T00:00 Beijing 50 50 true 50 50 50 50.5 50.5 char50 50 +51 2017-10-01 2017-10-01T00:00 Beijing 51 51 true 51 51 51 51.51 51.51 char51 51 +52 2017-10-01 2017-10-01T00:00 Beijing 52 52 true 52 52 52 52.52 52.52 char52 52 +53 2017-10-01 2017-10-01T00:00 Beijing 53 53 true 53 53 53 53.53 53.53 char53 53 +54 2017-10-01 2017-10-01T00:00 Beijing 54 54 true 54 54 54 54.54 54.54 char54 54 +55 2017-10-01 2017-10-01T00:00 Beijing 55 55 true 55 55 55 55.55 55.55 char55 55 +56 2017-10-01 2017-10-01T00:00 Beijing 56 56 true 56 56 56 56.56 56.56 char56 56 +57 2017-10-01 2017-10-01T00:00 Beijing 57 57 true 57 57 57 57.57 57.57 char57 57 +58 2017-10-01 2017-10-01T00:00 Beijing 58 58 true 58 58 58 58.58 58.58 char58 58 +59 2017-10-01 2017-10-01T00:00 Beijing 59 59 true 59 59 59 59.59 59.59 char59 59 +60 2017-10-01 2017-10-01T00:00 Beijing 60 60 true 60 60 60 60.6 60.6 char60 60 +61 2017-10-01 2017-10-01T00:00 Beijing 61 61 true 61 61 61 61.61 61.61 char61 61 +62 2017-10-01 2017-10-01T00:00 Beijing 62 62 true 62 62 62 62.62 62.62 char62 62 +63 2017-10-01 2017-10-01T00:00 Beijing 63 63 true 63 63 63 63.63 63.63 char63 63 +64 2017-10-01 2017-10-01T00:00 Beijing 64 64 true 64 64 64 64.64 64.64 char64 64 +65 2017-10-01 2017-10-01T00:00 Beijing 65 65 true 65 65 65 65.65 65.65 char65 65 +66 2017-10-01 2017-10-01T00:00 Beijing 66 66 true 66 66 66 66.66 66.66 char66 66 +67 2017-10-01 2017-10-01T00:00 Beijing 67 67 true 67 67 67 67.67 67.67 char67 67 +68 2017-10-01 2017-10-01T00:00 Beijing 68 68 true 68 68 68 68.68 68.68 char68 68 +69 2017-10-01 2017-10-01T00:00 Beijing 69 69 true 69 69 69 69.69 69.69 char69 69 +70 2017-10-01 2017-10-01T00:00 Beijing 70 70 true 70 70 70 70.7 70.7 char70 70 +71 2017-10-01 2017-10-01T00:00 Beijing 71 71 true 71 71 71 71.71 71.71 char71 71 +72 2017-10-01 2017-10-01T00:00 Beijing 72 72 true 72 72 72 72.72 72.72 char72 72 +73 2017-10-01 2017-10-01T00:00 Beijing 73 73 true 73 73 73 73.73 73.73 char73 73 +74 2017-10-01 2017-10-01T00:00 Beijing 74 74 true 74 74 74 74.74 74.74 char74 74 +75 2017-10-01 2017-10-01T00:00 Beijing 75 75 true 75 75 75 75.75 75.75 char75 75 +76 2017-10-01 2017-10-01T00:00 Beijing 76 76 true 76 76 76 76.76 76.76 char76 76 +77 2017-10-01 2017-10-01T00:00 Beijing 77 77 true 77 77 77 77.77 77.77 char77 77 +78 2017-10-01 2017-10-01T00:00 Beijing 78 78 true 78 78 78 78.78 78.78 char78 78 +79 2017-10-01 2017-10-01T00:00 Beijing 79 79 true 79 79 79 79.79 79.79 char79 79 +80 2017-10-01 2017-10-01T00:00 Beijing 80 80 true 80 80 80 80.8 80.8 char80 80 +81 2017-10-01 2017-10-01T00:00 Beijing 81 81 true 81 81 81 81.81 81.81 char81 81 +82 2017-10-01 2017-10-01T00:00 Beijing 82 82 true 82 82 82 82.82 82.82 char82 82 +83 2017-10-01 2017-10-01T00:00 Beijing 83 83 true 83 83 83 83.83 83.83 char83 83 +84 2017-10-01 2017-10-01T00:00 Beijing 84 84 true 84 84 84 84.84 84.84 char84 84 +85 2017-10-01 2017-10-01T00:00 Beijing 85 85 true 85 85 85 85.85 85.85 char85 85 +86 2017-10-01 2017-10-01T00:00 Beijing 86 86 true 86 86 86 86.86 86.86 char86 86 +87 2017-10-01 2017-10-01T00:00 Beijing 87 87 true 87 87 87 87.87 87.87 char87 87 +88 2017-10-01 2017-10-01T00:00 Beijing 88 88 true 88 88 88 88.88 88.88 char88 88 +89 2017-10-01 2017-10-01T00:00 Beijing 89 89 true 89 89 89 89.89 89.89 char89 89 +90 2017-10-01 2017-10-01T00:00 Beijing 90 90 true 90 90 90 90.9 90.9 char90 90 +91 2017-10-01 2017-10-01T00:00 Beijing 91 91 true 91 91 91 91.91 91.91 char91 91 +92 2017-10-01 2017-10-01T00:00 Beijing 92 92 true 92 92 92 92.92 92.92 char92 92 +93 2017-10-01 2017-10-01T00:00 Beijing 93 93 true 93 93 93 93.93 93.93 char93 93 +94 2017-10-01 2017-10-01T00:00 Beijing 94 94 true 94 94 94 94.94 94.94 char94 94 +95 2017-10-01 2017-10-01T00:00 Beijing 95 95 true 95 95 95 95.95 95.95 char95 95 +96 2017-10-01 2017-10-01T00:00 Beijing 96 96 true 96 96 96 96.96 96.96 char96 96 +97 2017-10-01 2017-10-01T00:00 Beijing 97 97 true 97 97 97 97.97 97.97 char97 97 +98 2017-10-01 2017-10-01T00:00 Beijing 98 98 true 98 98 98 98.98 98.98 char98 98 +99 2017-10-01 2017-10-01T00:00 Beijing 99 99 true 99 99 99 99.99 99.99 char99 99 +100 2017-10-01 2017-10-01T00:00 \N \N \N \N \N \N \N \N \N \N \N -- !select_load4 -- 1 2017-10-01 2017-10-01T00:00 Beijing 1 1 true 1 1 1 1.1 1.1 char1 1 @@ -238,4 +418,94 @@ 8 2017-10-01 2017-10-01T00:00 Beijing 8 8 true 8 8 8 8.8 8.8 char8 8 9 2017-10-01 2017-10-01T00:00 Beijing 9 9 true 9 9 9 9.9 9.9 char9 9 10 2017-10-01 2017-10-01T00:00 Beijing 10 10 true 10 10 10 10.1 10.1 char10 10 +11 2017-10-01 2017-10-01T00:00 Beijing 11 11 true 11 11 11 11.11 11.11 char11 11 +12 2017-10-01 2017-10-01T00:00 Beijing 12 12 true 12 12 12 12.12 12.12 char12 12 +13 2017-10-01 2017-10-01T00:00 Beijing 13 13 true 13 13 13 13.13 13.13 char13 13 +14 2017-10-01 2017-10-01T00:00 Beijing 14 14 true 14 14 14 14.14 14.14 char14 14 +15 2017-10-01 2017-10-01T00:00 Beijing 15 15 true 15 15 15 15.15 15.15 char15 15 +16 2017-10-01 2017-10-01T00:00 Beijing 16 16 true 16 16 16 16.16 16.16 char16 16 +17 2017-10-01 2017-10-01T00:00 Beijing 17 17 true 17 17 17 17.17 17.17 char17 17 +18 2017-10-01 2017-10-01T00:00 Beijing 18 18 true 18 18 18 18.18 18.18 char18 18 +19 2017-10-01 2017-10-01T00:00 Beijing 19 19 true 19 19 19 19.19 19.19 char19 19 +20 2017-10-01 2017-10-01T00:00 Beijing 20 20 true 20 20 20 20.2 20.2 char20 20 +21 2017-10-01 2017-10-01T00:00 Beijing 21 21 true 21 21 21 21.21 21.21 char21 21 +22 2017-10-01 2017-10-01T00:00 Beijing 22 22 true 22 22 22 22.22 22.22 char22 22 +23 2017-10-01 2017-10-01T00:00 Beijing 23 23 true 23 23 23 23.23 23.23 char23 23 +24 2017-10-01 2017-10-01T00:00 Beijing 24 24 true 24 24 24 24.24 24.24 char24 24 +25 2017-10-01 2017-10-01T00:00 Beijing 25 25 true 25 25 25 25.25 25.25 char25 25 +26 2017-10-01 2017-10-01T00:00 Beijing 26 26 true 26 26 26 26.26 26.26 char26 26 +27 2017-10-01 2017-10-01T00:00 Beijing 27 27 true 27 27 27 27.27 27.27 char27 27 +28 2017-10-01 2017-10-01T00:00 Beijing 28 28 true 28 28 28 28.28 28.28 char28 28 +29 2017-10-01 2017-10-01T00:00 Beijing 29 29 true 29 29 29 29.29 29.29 char29 29 +30 2017-10-01 2017-10-01T00:00 Beijing 30 30 true 30 30 30 30.3 30.3 char30 30 +31 2017-10-01 2017-10-01T00:00 Beijing 31 31 true 31 31 31 31.31 31.31 char31 31 +32 2017-10-01 2017-10-01T00:00 Beijing 32 32 true 32 32 32 32.32 32.32 char32 32 +33 2017-10-01 2017-10-01T00:00 Beijing 33 33 true 33 33 33 33.33 33.33 char33 33 +34 2017-10-01 2017-10-01T00:00 Beijing 34 34 true 34 34 34 34.34 34.34 char34 34 +35 2017-10-01 2017-10-01T00:00 Beijing 35 35 true 35 35 35 35.35 35.35 char35 35 +36 2017-10-01 2017-10-01T00:00 Beijing 36 36 true 36 36 36 36.36 36.36 char36 36 +37 2017-10-01 2017-10-01T00:00 Beijing 37 37 true 37 37 37 37.37 37.37 char37 37 +38 2017-10-01 2017-10-01T00:00 Beijing 38 38 true 38 38 38 38.38 38.38 char38 38 +39 2017-10-01 2017-10-01T00:00 Beijing 39 39 true 39 39 39 39.39 39.39 char39 39 +40 2017-10-01 2017-10-01T00:00 Beijing 40 40 true 40 40 40 40.4 40.4 char40 40 +41 2017-10-01 2017-10-01T00:00 Beijing 41 41 true 41 41 41 41.41 41.41 char41 41 +42 2017-10-01 2017-10-01T00:00 Beijing 42 42 true 42 42 42 42.42 42.42 char42 42 +43 2017-10-01 2017-10-01T00:00 Beijing 43 43 true 43 43 43 43.43 43.43 char43 43 +44 2017-10-01 2017-10-01T00:00 Beijing 44 44 true 44 44 44 44.44 44.44 char44 44 +45 2017-10-01 2017-10-01T00:00 Beijing 45 45 true 45 45 45 45.45 45.45 char45 45 +46 2017-10-01 2017-10-01T00:00 Beijing 46 46 true 46 46 46 46.46 46.46 char46 46 +47 2017-10-01 2017-10-01T00:00 Beijing 47 47 true 47 47 47 47.47 47.47 char47 47 +48 2017-10-01 2017-10-01T00:00 Beijing 48 48 true 48 48 48 48.48 48.48 char48 48 +49 2017-10-01 2017-10-01T00:00 Beijing 49 49 true 49 49 49 49.49 49.49 char49 49 +50 2017-10-01 2017-10-01T00:00 Beijing 50 50 true 50 50 50 50.5 50.5 char50 50 +51 2017-10-01 2017-10-01T00:00 Beijing 51 51 true 51 51 51 51.51 51.51 char51 51 +52 2017-10-01 2017-10-01T00:00 Beijing 52 52 true 52 52 52 52.52 52.52 char52 52 +53 2017-10-01 2017-10-01T00:00 Beijing 53 53 true 53 53 53 53.53 53.53 char53 53 +54 2017-10-01 2017-10-01T00:00 Beijing 54 54 true 54 54 54 54.54 54.54 char54 54 +55 2017-10-01 2017-10-01T00:00 Beijing 55 55 true 55 55 55 55.55 55.55 char55 55 +56 2017-10-01 2017-10-01T00:00 Beijing 56 56 true 56 56 56 56.56 56.56 char56 56 +57 2017-10-01 2017-10-01T00:00 Beijing 57 57 true 57 57 57 57.57 57.57 char57 57 +58 2017-10-01 2017-10-01T00:00 Beijing 58 58 true 58 58 58 58.58 58.58 char58 58 +59 2017-10-01 2017-10-01T00:00 Beijing 59 59 true 59 59 59 59.59 59.59 char59 59 +60 2017-10-01 2017-10-01T00:00 Beijing 60 60 true 60 60 60 60.6 60.6 char60 60 +61 2017-10-01 2017-10-01T00:00 Beijing 61 61 true 61 61 61 61.61 61.61 char61 61 +62 2017-10-01 2017-10-01T00:00 Beijing 62 62 true 62 62 62 62.62 62.62 char62 62 +63 2017-10-01 2017-10-01T00:00 Beijing 63 63 true 63 63 63 63.63 63.63 char63 63 +64 2017-10-01 2017-10-01T00:00 Beijing 64 64 true 64 64 64 64.64 64.64 char64 64 +65 2017-10-01 2017-10-01T00:00 Beijing 65 65 true 65 65 65 65.65 65.65 char65 65 +66 2017-10-01 2017-10-01T00:00 Beijing 66 66 true 66 66 66 66.66 66.66 char66 66 +67 2017-10-01 2017-10-01T00:00 Beijing 67 67 true 67 67 67 67.67 67.67 char67 67 +68 2017-10-01 2017-10-01T00:00 Beijing 68 68 true 68 68 68 68.68 68.68 char68 68 +69 2017-10-01 2017-10-01T00:00 Beijing 69 69 true 69 69 69 69.69 69.69 char69 69 +70 2017-10-01 2017-10-01T00:00 Beijing 70 70 true 70 70 70 70.7 70.7 char70 70 +71 2017-10-01 2017-10-01T00:00 Beijing 71 71 true 71 71 71 71.71 71.71 char71 71 +72 2017-10-01 2017-10-01T00:00 Beijing 72 72 true 72 72 72 72.72 72.72 char72 72 +73 2017-10-01 2017-10-01T00:00 Beijing 73 73 true 73 73 73 73.73 73.73 char73 73 +74 2017-10-01 2017-10-01T00:00 Beijing 74 74 true 74 74 74 74.74 74.74 char74 74 +75 2017-10-01 2017-10-01T00:00 Beijing 75 75 true 75 75 75 75.75 75.75 char75 75 +76 2017-10-01 2017-10-01T00:00 Beijing 76 76 true 76 76 76 76.76 76.76 char76 76 +77 2017-10-01 2017-10-01T00:00 Beijing 77 77 true 77 77 77 77.77 77.77 char77 77 +78 2017-10-01 2017-10-01T00:00 Beijing 78 78 true 78 78 78 78.78 78.78 char78 78 +79 2017-10-01 2017-10-01T00:00 Beijing 79 79 true 79 79 79 79.79 79.79 char79 79 +80 2017-10-01 2017-10-01T00:00 Beijing 80 80 true 80 80 80 80.8 80.8 char80 80 +81 2017-10-01 2017-10-01T00:00 Beijing 81 81 true 81 81 81 81.81 81.81 char81 81 +82 2017-10-01 2017-10-01T00:00 Beijing 82 82 true 82 82 82 82.82 82.82 char82 82 +83 2017-10-01 2017-10-01T00:00 Beijing 83 83 true 83 83 83 83.83 83.83 char83 83 +84 2017-10-01 2017-10-01T00:00 Beijing 84 84 true 84 84 84 84.84 84.84 char84 84 +85 2017-10-01 2017-10-01T00:00 Beijing 85 85 true 85 85 85 85.85 85.85 char85 85 +86 2017-10-01 2017-10-01T00:00 Beijing 86 86 true 86 86 86 86.86 86.86 char86 86 +87 2017-10-01 2017-10-01T00:00 Beijing 87 87 true 87 87 87 87.87 87.87 char87 87 +88 2017-10-01 2017-10-01T00:00 Beijing 88 88 true 88 88 88 88.88 88.88 char88 88 +89 2017-10-01 2017-10-01T00:00 Beijing 89 89 true 89 89 89 89.89 89.89 char89 89 +90 2017-10-01 2017-10-01T00:00 Beijing 90 90 true 90 90 90 90.9 90.9 char90 90 +91 2017-10-01 2017-10-01T00:00 Beijing 91 91 true 91 91 91 91.91 91.91 char91 91 +92 2017-10-01 2017-10-01T00:00 Beijing 92 92 true 92 92 92 92.92 92.92 char92 92 +93 2017-10-01 2017-10-01T00:00 Beijing 93 93 true 93 93 93 93.93 93.93 char93 93 +94 2017-10-01 2017-10-01T00:00 Beijing 94 94 true 94 94 94 94.94 94.94 char94 94 +95 2017-10-01 2017-10-01T00:00 Beijing 95 95 true 95 95 95 95.95 95.95 char95 95 +96 2017-10-01 2017-10-01T00:00 Beijing 96 96 true 96 96 96 96.96 96.96 char96 96 +97 2017-10-01 2017-10-01T00:00 Beijing 97 97 true 97 97 97 97.97 97.97 char97 97 +98 2017-10-01 2017-10-01T00:00 Beijing 98 98 true 98 98 98 98.98 98.98 char98 98 +99 2017-10-01 2017-10-01T00:00 Beijing 99 99 true 99 99 99 99.99 99.99 char99 99 +100 2017-10-01 2017-10-01T00:00 \N \N \N \N \N \N \N \N \N \N \N diff --git a/regression-test/data/external_table_p0/hive/test_partial_update_broker_load.out b/regression-test/data/external_table_p0/hive/test_partial_update_broker_load.out new file mode 100644 index 000000000000000..c12447f46b12ac9 --- /dev/null +++ b/regression-test/data/external_table_p0/hive/test_partial_update_broker_load.out @@ -0,0 +1,16 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 alice 1000 123 1 +2 bob 2000 223 2 +3 tom 3000 323 3 + +-- !sql -- +1 alice 200 123 1 +2 bob 400 223 2 +3 tom 3000 323 3 + +-- !sql -- +1 alice 999 123 1 +2 bob 400 223 2 +3 tom 888 323 3 + diff --git a/regression-test/data/external_table_p0/jdbc/test_sqlserver_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_sqlserver_jdbc_catalog.out index d689f4cf6cbdffd..c8053812c1b6196 100644 --- a/regression-test/data/external_table_p0/jdbc/test_sqlserver_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_sqlserver_jdbc_catalog.out @@ -53,3 +53,6 @@ -- !filter3 -- 1 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! +-- !id -- +2 + diff --git a/regression-test/data/external_table_p0/tvf/comp_arr.orc b/regression-test/data/external_table_p0/tvf/comp_arr.orc new file mode 100644 index 000000000000000..d264c01089aadf3 Binary files /dev/null and b/regression-test/data/external_table_p0/tvf/comp_arr.orc differ diff --git a/regression-test/data/external_table_p0/tvf/comp_arr.parquet b/regression-test/data/external_table_p0/tvf/comp_arr.parquet new file mode 100644 index 000000000000000..903f393e76228f8 Binary files /dev/null and b/regression-test/data/external_table_p0/tvf/comp_arr.parquet differ diff --git a/regression-test/data/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.out b/regression-test/data/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.out index 932b27b65c2d787..ee106e32c8e035e 100644 --- a/regression-test/data/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.out +++ b/regression-test/data/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.out @@ -78,3 +78,129 @@ -- !sql -- 0 +-- !sql -- +0 + +-- !sql_orc_tvf -- +0 [[0.075146018909447831, 0.83184654292916682, 0.48344758887617112, 0.73818042587686938, 0.1157914153224413, 0.010906227661216183, 0.14399412358433106, 0.44089849151182114, 0.57355556205718361, 0.25312517135856316, 0.38150647140194149, 0.070397728596659159, 0.86953536344447424, 0.68971873285317431, 0.43295999037537725, 0.6784883106064814, 0.59748020017550263, 0.701551888426747, 0.59172895985772778, 0.34453720788885822, 0.87415544636692688, 0.32542734889457547, 0.15307756621895752, 0.5967639742587556, 0.27981005098527589, 0.2878085040368038, 0.47320794944932354, 0.462573273179657, 0.181991720511385, 0.65965994183508947], [0.30669042061243168, 0.40772045173026605, 0.79942953279686146, 0.79112745662075734, 0.9162031241950418, 0.47977049680735451, 0.70743053959785085, 0.88106846902940883, 0.058859311770482536, 0.5582244050439672, 0.61221349612133946, 0.26398328723774411, 0.845002934668909, 0.7318513641061144, 0.12882949870214055, 0.25568561677314683, 0.63820374314148409, 0.47317928430060152, 0.98579534589233231, 0.18909131775818011, 0.61928522310719925, 0.46076003147362854, 0.54929286054482063, 0.092791921536963917, 0.37257051222485726, 0.61173970164373448, 0.81076330028131915, 0.31761967958249016, 0.86913228888996208, 0.1304288619260594], [0.01395281464511211, 0.30019929214213881, 0.51968481559571245, 0.1529812070388713, 0.91094127854135332, 0.56752084306826045, 0.54885745773319083, 0.87395325967138593, 0.23141383104711721, 0.032857330783252037, 0.80294086357636207, 0.87743211429809975, 0.32541037759757729, 0.610889338006772, 0.24479853782135153, 0.76358175946904172, 0.097015496504703536, 0.18868863101151945, 0.812487590812745, 0.060873224844278262, 0.28492733296189487, 0.029251182023422917, 0.022235039951404745, 0.84492712327496655, 0.86557067014930533, 0.75555932392457192, 0.68176625973315275, 0.44681195608518232, 0.85539631857096821, 0.16569204251912195], [0.34658514841506916, 0.1268049901580286, 0.071234875518197849, 0.26329111773144742, 0.69965586352226594, 0.61403698266195006, 0.47105010114528778, 0.052729894896325624, 0.86403861188530129, 0.96810607785609748, 0.542163311482091, 0.84462116899381379, 0.76838267110247349, 0.30565975588120686, 0.82999691111906371, 0.044838509782801461, 0.076782055123802762, 0.55310856391411212, 0.10761171932456115, 0.65901588915115827, 0.86735027524090058, 0.53604895744114833, 0.87478938101481474, 0.034528135493820389, 0.19797273921449776, 0.396636369100942, 0.84513740655578207, 0.68324497247689842, 0.9330462106026346, 0.54206449863011907], [0.83067163764091867, 0.3808895221813462, 0.72161718918340434, 0.070470082806093348, 0.9655564635473981, 0.1226220247446318, 0.81518155502263967, 0.894996860337646, 0.52416087663426092, 0.92950769116382537, 0.49251073906158571, 0.59241651388338123, 0.68331376414364187, 0.94681786268464219, 0.21921961806975321, 0.33961833290240029, 0.45461966218843741, 0.75265132021366643, 0.31139451010915242, 0.16944322666300871, 0.669921659675672, 0.30415288586162126, 0.795450048218866, 0.176752390721302, 0.341475273039066, 0.48864460278142685, 0.3604857903006139, 0.23628415735788122, 0.93812387557784538, 0.8182746134054657]] [{"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}] +1 [[0.36347909454423355, 0.41846626684948152, 0.079114592175514309, 0.66786028640635076, 0.3937097712237253, 0.789970132134358, 0.76656970664847057, 0.33365439648974671, 0.82537818944612018, 0.67177137061591408, 0.27802211322330717, 0.30797446380318649, 0.48919046879622785, 0.026597605734909657, 0.86028930790459013, 0.59862131876428393, 0.63368577900374445, 0.5431679589950682, 0.65036371203785692, 0.44948614963712841, 0.28018625320876811, 0.51570991706205271, 0.025688287032002211, 0.76269923674189588, 0.68635645089977859, 0.16699308447773564, 0.92652113120235846, 0.13746724514453723, 0.28522352481007851, 0.074944393436347845], [0.22280686747520972, 0.52013661953979584, 0.54170574612594191, 0.75841707425706861, 0.22629378640520526, 0.93365330530754753, 0.40014150143354366, 0.27944412692112619, 0.1332052036540543, 0.13447163712493426, 0.38400646685948181, 0.92731394014633517, 0.52359455831396784, 0.33109823808979311, 0.95295239146355981, 0.66406449231411657, 0.040583333692196466, 0.29083957690919393, 0.42249539731446362, 0.031499483050802746, 0.026001816228967467, 0.2599545277374522, 0.60812777394085393, 0.64658816796002283, 0.19852280082713492, 0.12729620943748521, 0.36832215328736362, 0.24511836673200982, 0.42619840669062325, 0.1726601907599643], [0.38984244135290436, 0.47272954855476157, 0.2673810361664567, 0.5584386712736088, 0.88397101557440194, 0.26796130521751726, 0.2323292380097145, 0.4468137216335466, 0.34370717309715526, 0.73370993457568, 0.65236991210468709, 0.25460472165830428, 0.14968718730862707, 0.85149366094614021, 0.63430855897318417, 0.094018303897087208, 0.60652131528355657, 0.25718503570897666, 0.82484947710904366, 0.8899321720123512, 0.94143878487960542, 0.56737682896975206, 0.61310816781385513, 0.012976431924834775, 0.90357683602979455, 0.42230539059768024, 0.95886849725530243, 0.52255921819927231, 0.86494554799831258, 0.49426339882785542], [0.250057791257716, 0.27511637550765422, 0.743044784069812, 0.19969166496450474, 0.4649825842540799, 0.2202471542455382, 0.64030594364852622, 0.74864758117472163, 0.44318905600166236, 0.18103153566971941, 0.826975499454017, 0.46710551238932208, 0.25338160240396523, 0.29470675622636755, 0.58175335891467672, 0.035552091940969643, 0.48765397206038663, 0.07601413130299417, 0.44844392884487205, 0.40159769902524911, 0.80835727577283889, 0.75790953934354222, 0.66707324369196874, 0.644832586228614, 0.22724956640213656, 0.1629642575957917, 0.89437113341594165, 0.99891436279760737, 0.40614244217464945, 0.86366506514462094], [0.84121189081814984, 0.56888116999757854, 0.73407954905913542, 0.800955852695311, 0.1235533940121567, 0.020942877401750382, 0.058153150914470486, 0.84774561366867862, 0.44860858797286907, 0.66750999978632219, 0.72338456817599628, 0.027740412414764992, 0.23943000902995171, 0.92197855672246887, 0.011664804999946043, 0.22983961353479565, 0.93878194064271059, 0.055796222853087718, 0.16057216714992129, 0.294470205491835, 0.52712115242213, 0.55459333471484129, 0.8899536416494972, 0.29061466047068207, 0.23970198873913651, 0.14066987147744947, 0.76747753093944149, 0.65092177705500376, 0.0742669841436645, 0.70588781283143143], [0.80393477414713888, 0.58717195688191814, 0.44347928124587233, 0.98202533050932206, 0.38488675309367737, 0.86950136614463114, 0.24519926868457564, 0.76490786218976625, 0.017328170830664957, 0.62017283159584224, 0.69890177897433126, 0.90413828247774619, 0.39835315178533226, 0.24631301749474188, 0.52797686536944943, 0.072006896893512584, 0.99738354839389343, 0.015919164929985796, 0.45535157681805349, 0.53316887600648588, 0.68433867187319986, 0.067571333797358912, 0.52381605549709642, 0.75093546018670243, 0.93319574155424212, 0.83666838566966917, 0.45263195050630989, 0.68653264186914253, 0.78371105773959437, 0.97963941768735019], [0.098447300967908546, 0.07443874066552536, 0.05364162312387688, 0.88246475334594587, 0.10123641377404269, 0.789920401619719, 0.25962818572797042, 0.75097359949233533, 0.96492943672532139, 0.66414405510781049, 0.32696974165492931, 0.0977680468839287, 0.14759093914880117, 0.747748223744763, 0.44684103813826492, 0.164256312098906, 0.21745266069093849, 0.075518803386006583, 0.37407422008360369, 0.446160635405205, 0.76880054398410858, 0.44281888833985961, 0.049446193515169434, 0.02699795647106451, 0.585661584290046, 0.56868059674765481, 0.67748385157567259, 0.86641471535604575, 0.49143150669862756, 0.29097602701531011], [0.29411933753860242, 0.53888949915174611, 0.42902938687211079, 0.11584074293199953, 0.41809785404179434, 0.17320039917247598, 0.57886051170810582, 0.52653213826344814, 0.59586110883584986, 0.504186608030426, 0.74081473200375825, 0.946162171008541, 0.52169942069082653, 0.015716372598683193, 0.78172190605308511, 0.14425186105926702, 0.94328618052761326, 0.12740691124548464, 0.68113231757547243, 0.17881324017649514, 0.747278356726216, 0.25752393252970207, 0.59598441836341853, 0.67367153556345361, 0.46048722710355883, 0.38062150587929666, 0.4086683973245484, 0.344998612268513, 0.899193991707704, 0.3908662044512391], [0.423188396502708, 0.21153661996401996, 0.85250472959367374, 0.18062171271885152, 0.23057264490443741, 0.849748267119177, 0.49520988865142479, 0.84875310547841587, 0.27737504119357914, 0.38813791036924117, 0.69143405256418478, 0.86424295624850977, 0.34005760124547024, 0.67424626522471809, 0.47243104020173665, 0.32809403833080852, 0.93323877392329047, 0.19332793162241235, 0.75020100667466583, 0.700126356365772, 0.661236755962131, 0.86948025489808578, 0.76317546782863477, 0.48715890459730482, 0.66003400036375681, 0.8379988110295854, 0.84103703975244659, 0.42519956080937527, 0.57073267491522106, 0.64199294836620591], [0.82281096653815, 0.63064910225319115, 0.93576745975954956, 0.076802861309765436, 0.86568542658314207, 0.044158261607361693, 0.51558653015789169, 0.082516973795488169, 0.441588531069972, 0.2456596843788561, 0.78620750347433854, 0.66789690908724575, 0.034369637626316507, 0.38022920182811903, 0.79699996008614349, 0.83827426063719435, 0.422503481741183, 0.501813770347819, 0.64324159000688563, 0.89549206462213859, 0.46055523450336167, 0.11549639161351744, 0.039440817188449273, 0.2797212237176987, 0.957215848262082, 0.078850691907273784, 0.8671130341932527, 0.3309897083284602, 0.58853639051021567, 0.30907140683236778]] [{"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}] +2 [[0.877221380588093, 0.34747559807707384, 0.3675645915876965, 0.52652201116175712, 0.72002970080866924, 0.56105762000298276, 0.0643113918615007, 0.24286385119102094, 0.93193769482862376, 0.73982672018891238, 0.32241219536777865, 0.23776429755650352, 0.34347912299009942, 0.088746649808589928, 0.87220220991248354, 0.39688406891819517, 0.78830357499627624, 0.14570546857506361, 0.76341782137652059, 0.80827256407163572, 0.26071189588204635, 0.0510259473418454, 0.547961177300181, 0.26710245964353851, 0.97103885995839045, 0.57969895488720968, 0.11937708742100628, 0.91930231085860548, 0.56392810599871812, 0.28170721402208376], [0.61324778199729268, 0.25127080800817936, 0.64442747842796988, 0.96350229243099417, 0.48071892107803582, 0.473677028506853, 0.54296545146173514, 0.78764268265190707, 0.96162713582418169, 0.844420885129845, 0.94265218479617574, 0.46779654521973335, 0.70420858480105386, 0.62914880822235431, 0.59098185136086878, 0.3927576640248025, 0.20512469587153759, 0.20728962091476633, 0.82289522124031844, 0.025818600365801303, 0.076549446336385607, 0.29739595747247438, 0.033294705372403754, 0.59479052967517143, 0.88632067547226312, 0.39764710803066761, 0.55448657852034, 0.91315407230686885, 0.46266508583671029, 0.31568913285697842], [0.61765820968986385, 0.15427146464800578, 0.10676508232206716, 0.096558727645921882, 0.20738062263124291, 0.53828476832197369, 0.11444276854086888, 0.74825728217292153, 0.65425987082622938, 0.18527128318725972, 0.37249489131766478, 0.91161251195130255, 0.75324288638432779, 0.83842159286660023, 0.56571961363761714, 0.024384348639726383, 0.04371752648483529, 0.76745149965912385, 0.70743452068006563, 0.58914321983392881, 0.64242759191901944, 0.92515456174278132, 0.28929261484228885, 0.87026831982481456, 0.681358126816789, 0.34894680604821759, 0.33546079855688604, 0.71748411405996537, 0.97012788948265827, 0.87445078128165665], [0.041733925260367566, 0.2416737455994501, 0.97605941389874418, 0.45961152832251628, 0.48191384944148596, 0.75327793868741355, 0.59497952004106869, 0.80553195179579118, 0.071792599319834949, 0.7144088680809102, 0.47193601351918824, 0.504160967135871, 0.09533484173370721, 0.45287812231846114, 0.82339938786838152, 0.64316243674320406, 0.61816382436668382, 0.039658793364443556, 0.81115074183095393, 0.65473896419593769, 0.54021227164917462, 0.0408716239563377, 0.52294022006549823, 0.2078792109912514, 0.37240734879552406, 0.82120578094590679, 0.52613299113540057, 0.11244315855991893, 0.56151756833974131, 0.69775466314298218], [0.77794265759464531, 0.52869506489843954, 0.832795789132383, 0.52156814769376469, 0.39948481109146172, 0.2573388346016795, 0.071811762487268616, 0.19879235626169789, 0.45076590262533889, 0.52280611683199185, 0.72487463470033553, 0.25408864620450688, 0.37985633354816595, 0.45649286316554216, 0.56170383186996564, 0.78609388066071006, 0.26187816992166346, 0.43683491398169838, 0.91451471424145414, 0.068004045368166732, 0.29207182039549995, 0.54955898782761292, 0.18444277163728362, 0.60593412142657888, 0.25208843214463261, 0.6941405987709649, 0.48923874196003714, 0.68352672082392874, 0.82978720032126108, 0.28144550329452567], [0.81542386123156052, 0.29335313189073764, 0.74020035662224692, 0.042727070548302382, 0.93360299793324653, 0.88396984157148062, 0.56896075384837563, 0.20159674922380932, 0.63739098942789407, 0.53606478738120333, 0.13936996071392138, 0.14828055280494779, 0.57441532076986312, 0.021947161583771191, 0.99041799880484671, 0.79941962295795932, 0.052115859607728021, 0.69970558332311894, 0.24544796527828305, 0.026867935186114522, 0.72157093684986384, 0.3466788760269619, 0.74979211522584277, 0.89538643759887049, 0.96933558656034036, 0.84645521797049672, 0.86058860433900985, 0.96182683115498335, 0.1802871475633373, 0.10943452858148861], [0.68312098419182976, 0.55734514265019552, 0.77372978259920266, 0.72905991832340067, 0.8654053265653282, 0.79937892174260883, 0.060518197715768274, 0.65414452632757192, 0.28638236586677668, 0.061568524226932042, 0.520718502726366, 0.59228973973100485, 0.10295558624609502, 0.76749759329807821, 0.059331517881558771, 0.9502113902501228, 0.65724816458046453, 0.66807314128948025, 0.16186224348939016, 0.86386141613295742, 0.68783509231885709, 0.21256131301222791, 0.64202673092804219, 0.274551649625869, 0.13733928386718697, 0.2297398309683113, 0.85064517899945891, 0.066857656163357593, 0.60366346726365294, 0.89832383208563948], [0.79777853353547279, 0.99990042476906427, 0.14326228007519537, 0.90476656178823045, 0.62942294051390479, 0.53567423110177748, 0.70224210910604423, 0.76173251875319026, 0.29558476097885877, 0.69640901780273745, 0.92470874376446854, 0.49198502827412061, 0.048316354019785179, 0.76752627888685965, 0.46766572405140405, 0.27012621650968471, 0.61630088755815371, 0.0035675335801024222, 0.23789929550999545, 0.15857244374998358, 0.51683201753864, 0.491544849801956, 0.15866865827107446, 0.77434351996690831, 0.986542188986288, 0.9794832602050173, 0.17265268574389447, 0.5796893511184591, 0.43899128905126483, 0.7874950360919506], [0.19248748891184131, 0.50227313935811357, 0.19023479676981214, 0.88454634698837709, 0.62136193740762424, 0.7595752288076052, 0.43274884243411038, 0.83614695698989527, 0.88352078678250068, 0.087377402582469954, 0.0835970571561504, 0.60651391907126029, 0.1281582967190894, 0.053632614814570712, 0.18448204193117923, 0.13333082965388832, 0.90006765234999686, 0.77672000701194577, 0.98933216391191359, 0.18404392068255304, 0.1065445359451247, 0.16103139937307487, 0.73933216589315964, 0.24970737514108454, 0.052426004554130934, 0.71302952443856749, 0.395813529660335, 0.13570981881129141, 0.66377857464983958, 0.10736443385741867], [0.27817921459540373, 0.46828475266636571, 0.27563209741345041, 0.93532980863742821, 0.63704408110911637, 0.55152769022944481, 0.52028914957381789, 0.48688539864478508, 0.68203113170081731, 0.88171339793242565, 0.92012223648073721, 0.99266847364726574, 0.47917078643455269, 0.15402659267738084, 0.76240401532691815, 0.48386081515603252, 0.43460876439733354, 0.78138993642706189, 0.71359829678057629, 0.31547488281662284, 0.789335488394516, 0.78589372079599418, 0.41667292013973112, 0.44256667541016215, 0.66390802507538949, 0.42040133140758817, 0.0073779472321962025, 0.73907864535405687, 0.7126434929971982, 0.03122555309154329]] [{"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}] +3 [[0.20833707059306106, 0.26596035255300954, 0.63563007761753176, 0.85541770480854418, 0.40115034778714709, 0.9412613207041074, 0.8295909849011508, 0.48000303149949053, 0.6118879191280987, 0.67759497996881635, 0.92409526089492067, 0.84484713127582289, 0.652624205322621, 0.055773066161040386, 0.51631413723903052, 0.70729443705654438, 0.90360969475785524, 0.17993538649176455, 0.94139391471764133, 0.70606291612943428, 0.26768973741783764, 0.39085624648091366, 0.9210279883272956, 0.99311846433880391, 0.802954368497856, 0.46550326396845243, 0.0757001581009702, 0.54957112369131755, 0.2738844415126529, 0.60878805117310986], [0.62728990184283973, 0.81623919067527806, 0.29192271943186621, 0.26317340679580981, 0.69001936736085923, 0.25487814329356517, 0.67831319141409541, 0.26277926733233115, 0.65026082496178572, 0.72267361580555656, 0.17521545727071219, 0.1408557116335466, 0.26951266783223082, 0.090448619842890654, 0.65551548902295331, 0.46296786659319822, 0.30989043226643787, 0.36413711388433412, 0.53741797850714068, 0.90296456504791944, 0.67776936902221241, 0.67942750404063745, 0.19681939307836271, 0.404778450216702, 0.30875315080402554, 0.931081120042446, 0.7886251903939635, 0.18033239278114288, 0.27593317858514466, 0.47128250887094814], [0.037588568460870109, 0.48230362006032013, 0.89344135803213254, 0.265479526244196, 0.23991225955273732, 0.060222941072004432, 0.9085888388941884, 0.12952336700313094, 0.92876867980231381, 0.95936367473445128, 0.22842185439589902, 0.18851971356307884, 0.40005821862912638, 0.85410120528303912, 0.010315133355070283, 0.49565384044471594, 0.56913928237035361, 0.42090052733362404, 0.36126213323988854, 0.82835462856045317, 0.43492777371213664, 0.35081582212898277, 0.33681082428040743, 0.039512914344140948, 0.76467760533934714, 0.47736268362938872, 0.40813234863403025, 0.98307228537468461, 0.4809184769013668, 0.15285028196831463], [0.074499372612602643, 0.95616144665452452, 0.63144032826133878, 0.44473688654070653, 0.8384895708717538, 0.80403857221409092, 0.753878852608626, 0.94329935114731633, 0.33251914287478235, 0.31947050439179003, 0.90836975329943892, 0.97288318381235639, 0.98785023767030911, 0.334770094038498, 0.64490405338219436, 0.78122672558232886, 0.271004876582543, 0.95256757624221333, 0.69643048148794051, 0.844829374894464, 0.17755750259009717, 0.69049278856170682, 0.24441514793013974, 0.77306035772369908, 0.36956714494655774, 0.9252968102962692, 0.19048288382274348, 0.41968121397406, 0.070449157735003376, 0.91354311441490565], [0.98592838616296719, 0.717681589845808, 0.58987563115403718, 0.83884998075391093, 0.22029973621675158, 0.44180460082325745, 0.14133286950346557, 0.78976749644926436, 0.35092070039420553, 0.57286443894912253, 0.43625278169573989, 0.307723773584615, 0.66713215266162151, 0.18893589054579252, 0.39117484403652214, 0.16107680466600094, 0.826197556551212, 0.84072112896576057, 0.16818384838988032, 0.29916017694370889, 0.07223086525556488, 0.18187684445916408, 0.68390787882160853, 0.51931163370854638, 0.55516957099699, 0.21981032731322125, 0.668649853119051, 0.24022602081373334, 0.0037297677495421455, 0.92843021163308415], [0.33476558020002334, 0.91657503274522634, 0.37444402916704356, 0.12156418893834986, 0.72218937027163455, 0.052687863076487651, 0.99829701876291055, 0.42423218764999182, 0.82904850207115355, 0.47718875815089, 0.17713760986335292, 0.89869044882935056, 0.70659568125556516, 0.52583855705715421, 0.14396083738770316, 0.38313832870421194, 0.800228754573383, 0.046429229429328478, 0.7943165184647224, 0.88320600141404693, 0.95215533763694626, 0.68743174990021461, 0.62051571201569367, 0.41981866841660842, 0.64629923291554581, 0.76119952155523185, 0.83597039995080979, 0.58175785764491117, 0.27233470962312623, 0.082880549961004246], [0.763915402598785, 0.657218831548153, 0.41327372456620282, 0.57779383494785075, 0.18292607853689435, 0.625888164058225, 0.41982013175964361, 0.12287653061623371, 0.47105907557506466, 0.48154289943983908, 0.82474408222750983, 0.041910142955758012, 0.0711075916742423, 0.32489918205847212, 0.47181875155186703, 0.9095855098429988, 0.011376750505389333, 0.41741456059204063, 0.58640092196662919, 0.8360680466607453, 0.95954857266370686, 0.91897475525313188, 0.39852552151738818, 0.20157614630973697, 0.56162024119065057, 0.34587281530080682, 0.34442377316113115, 0.88958726120876841, 0.62356339748335687, 0.88021412583607461], [0.3186631945108318, 0.42763769447110389, 0.030266060887224278, 0.53785831327877565, 0.92664725139005033, 0.086283711321910372, 0.16448326183648088, 0.85923491841736666, 0.19600111300443002, 0.90964324998287294, 0.9879361242134701, 0.22584673473923, 0.74540926288342579, 0.85503893587075874, 0.87736337305276224, 0.12500096603317379, 0.98075951060499322, 0.70461461186199315, 0.7702431698502, 0.68628524171691363, 0.49098765519979914, 0.178399458786202, 0.16806772832277528, 0.59553158066673406, 0.44980501963307784, 0.72614501075152049, 0.069466326846047588, 0.055488626170968169, 0.49453144940436977, 0.80436205715803788], [0.9611374180294403, 0.80714842730182579, 0.24954415519904671, 0.055336060683558896, 0.581463741180078, 0.351651025181417, 0.38939298646089437, 0.69650129322675636, 0.28984900761297294, 0.41991629836589717, 0.89542074527330462, 0.60140508684237648, 0.52422830379075969, 0.4256296286188801, 0.94976717526409438, 0.0075958148105960133, 0.0098690335905130544, 0.10693379112209267, 0.73192354819418159, 0.52211914946793048, 0.094492465401062709, 0.81315668918320072, 0.23554416757507757, 0.11190912777256279, 0.82791096600129344, 0.92513060955043569, 0.10701043689268153, 0.064211879149065121, 0.17878891695628385, 0.71113585040873639], [0.422281520626668, 0.481281620740993, 0.086800859163047916, 0.5152272176059195, 0.2648553689258244, 0.73393968951577226, 0.88537277082707566, 0.550679452709282, 0.94520249303611514, 0.44018979155750848, 0.9983579610541331, 0.72851348343738109, 0.71082424592631865, 0.95793305789668881, 0.713412088649505, 0.88954243817960565, 0.83612616971362785, 0.084929060785173927, 0.26549645105299824, 0.52810237673103511, 0.052586142330613872, 0.64449986463431252, 0.42585253251564936, 0.5966098942061776, 0.38223624766871889, 0.9020762484105751, 0.28495473281167039, 0.26231497878293408, 0.571265999542488, 0.30553626555727265]] [{"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}] +4 [[0.047413350785556418, 0.86115446017354413, 0.52746594712888129, 0.7928220552172901, 0.378383677779879, 0.25869150807878805, 0.881336384872182, 0.43190601429121023, 0.34821786760165918, 0.60939011573389446, 0.57890524957964584, 0.12655679055526281, 0.98326350849592081, 0.4114752881455106, 0.71029502619961726, 0.17406205968209665, 0.4567325446056, 0.90624922448958, 0.375303240937634, 0.0988566852627778, 0.84170079910912943, 0.50945581118823879, 0.45747093315733334, 0.23290494502014036, 0.017642485218647308, 0.41312510078138509, 0.72675036381445246, 0.012354513367785591, 0.4402468598160616, 0.83449032844290816], [0.39116343053711355, 0.83473027751201345, 0.75086596885987145, 0.88173494855696688, 0.50006927291488479, 0.45472645423423164, 0.74979393980696951, 0.030740687327810545, 0.66513612030188252, 0.061693560805254122, 0.0866797027769014, 0.7626717454065175, 0.4098389459056091, 0.37820648027462078, 0.1317269948899964, 0.99914454034022682, 0.084280855716352354, 0.34663840683396707, 0.88293624789722747, 0.58989621584213137, 0.68936292517591768, 0.22945503664309197, 0.43749581574872465, 0.25727945751337289, 0.142803893749753, 0.23898194197162825, 0.5095299444282152, 0.082574869837416909, 0.48923726091599262, 0.13368765830603113], [0.71176397108376144, 0.83102659831568138, 0.30759017801845967, 0.57056343473027549, 0.83526775606625625, 0.15567065261330715, 0.1011751750762262, 0.38697030610078853, 0.15929045737020631, 0.99443972343121434, 0.76940208079065442, 0.40036274221478307, 0.90230102326709427, 0.28069209973022591, 0.35474409858952993, 0.760587387554492, 0.28570063614155516, 0.42422625445729434, 0.13456890649812214, 0.36699452061511451, 0.32327458169065448, 0.88800756582430329, 0.78467158733005515, 0.37138874488462426, 0.3723026411045296, 0.58524615450757, 0.779823373518607, 0.32788461068073715, 0.68634928842648146, 0.7521306371146339], [0.41637640248962271, 0.12202195593296539, 0.69510520269603415, 0.41343528560253817, 0.84294129471744583, 0.414034729724607, 0.50325461327878673, 0.55138287133618125, 0.26738092556915893, 0.96495611869585907, 0.14819369078492006, 0.612559336210704, 0.071840680962485637, 0.87928193325497173, 0.48552689813295713, 0.043340385007249793, 0.91226111171004487, 0.49420333854253506, 0.18162006235943817, 0.13120446073234093, 0.0047967339701571188, 0.3729587915178868, 0.18409015774936321, 0.54747023052045485, 0.70879029312371022, 0.61864436445592785, 0.21396343294667208, 0.63641108936748092, 0.20442861742058904, 0.51486079620275482], [0.27733537518360585, 0.676363120219015, 0.53293319280657259, 0.320864273012757, 0.92971705546238126, 0.050356983104870245, 0.51811452485369769, 0.90142721826526584, 0.0428544573738836, 0.50696841759555344, 0.12034618462716717, 0.75540483270182623, 0.12165847739161262, 0.46649586561365375, 0.5874912834614231, 0.9220413137218596, 0.79798540759164782, 0.63323800492929916, 0.36204009291734607, 0.1903395590560153, 0.98835869725140846, 0.011557952643679426, 0.079694602506116707, 0.014356776372921543, 0.34432467913055254, 0.79158495494945347, 0.15186206684329084, 0.54607769831614772, 0.046986726157523862, 0.46981843504470633], [0.36299633214011884, 0.237555984131546, 0.70136303596186533, 0.00056342957609856192, 0.63345224899351127, 0.53530360458762771, 0.071565919338863693, 0.73604031796679392, 0.59949486370827421, 0.292446915563335, 0.66440120510631473, 0.40617605370137932, 0.94433204913105673, 0.65795753214119135, 0.71683810344194154, 0.41349788841899171, 0.47751519377061613, 0.36730451428206567, 0.9719017582092474, 0.27421494119317591, 0.80477991391046544, 0.55516422586227765, 0.73789191662313558, 0.12348686357132133, 0.056374477453163818, 0.43836286371663513, 0.1174329184732037, 0.82104270304038252, 0.5491508894166196, 0.47601541345675191]] [{"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}] +5 [[0.81747151151055641, 0.24284083030137293, 0.27264751674096122, 0.68978250796757046, 0.56872724536845309, 0.51449315531515172, 0.053685801305193293, 0.76384965769711222, 0.86742444325808721, 0.17708620392635666, 0.20158720516586492, 0.35341653336535162, 0.37768603540715184, 0.30457216383931474, 0.070651854062912967, 0.66214881379723012, 0.43993158671841415, 0.088312991072720948, 0.55867249478793035, 0.18247298075352136, 0.4324783236970281, 0.1916294092482187, 0.16741542850851077, 0.56457588764789424, 0.84273796215882091, 0.17683757212820073, 0.24317123181410039, 0.37175013805346613, 0.45013937705148421, 0.68156216751123189], [0.038275147650006947, 0.71967438210963253, 0.19575872329921551, 0.67052783878173017, 0.92297321148144107, 0.58725688940704979, 0.7677403280493168, 0.26969568534113764, 0.54673710859106017, 0.40795937269940186, 0.66694309792287165, 0.4015254137978097, 0.99910676743138249, 0.40305759738808855, 0.53384888609503278, 0.68330729752846076, 0.052366942096037872, 0.0020839396039200597, 0.81562081111997442, 0.82492328098686174, 0.38183996124136321, 0.61073649430415244, 0.43130085276655106, 0.06805920930396947, 0.25430407358905949, 0.93282815360227311, 0.95348794460355168, 0.045805414198367478, 0.55102237653575625, 0.030388064313392915], [0.96643913598219777, 0.80974637795659887, 0.14909513809527342, 0.31481731555197712, 0.68338804187325275, 0.67105425283081033, 0.62332386854995214, 0.70203942496922578, 0.609976231087292, 0.932765395217531, 0.95050232816618274, 0.2358752587482944, 0.026044987983461, 0.55233270954276759, 0.39902672805489525, 0.58641130167627742, 0.32463634930216834, 0.26671100580335394, 0.0689430834048953, 0.96391922754647086, 0.79490367485687874, 0.73664658001091754, 0.611027066623565, 0.15392338760358615, 0.26473388433825085, 0.82250608167264661, 0.49821753393246559, 0.57530253643746221, 0.12923497344949031, 0.19089391375943265], [0.84870212801585843, 0.4908506750878866, 0.68284382392931942, 0.0694717693453003, 0.20717950291417464, 0.7585111578883692, 0.34566100135629996, 0.1168072680605905, 0.87929936991626756, 0.80153709496535164, 0.3287209386456148, 0.054743652190665681, 0.38400802437929871, 0.7971206592049872, 0.5182570954050405, 0.92269551568978381, 0.33950404315254978, 0.43557216324552894, 0.87562363040940394, 0.55644053327512477, 0.63073472696779831, 0.98591955477130155, 0.44738663428944325, 0.16839091246792026, 0.019840941266433321, 0.81883763466752157, 0.15919893780352612, 0.71199168812575975, 0.45568425590431305, 0.984105471571594], [0.92742521764681307, 0.049808356144718013, 0.21456711461977784, 0.91732975079343759, 0.26408460294328351, 0.81302921535245887, 0.66148319559160473, 0.8594602912075392, 0.47352995175072543, 0.23952238609369336, 0.85971142307291137, 0.11266382140883935, 0.74338494287235568, 0.20825159525545223, 0.47629431494654628, 0.82741050985376308, 0.9176106984008795, 0.14976862440434702, 0.43191726693314558, 0.618167047260183, 0.50617639300019857, 0.50566455536939536, 0.525932111960858, 0.93887924013456359, 0.53365894032337557, 0.93427584114169937, 0.83074509291650855, 0.814466017425174, 0.15819893655038597, 0.51560571814944312], [0.99451583140917943, 0.22083577599030091, 0.60259112711403184, 0.79533477066397085, 0.71709014310276831, 0.13347515442447822, 0.83089006984831648, 0.62595928621262331, 0.57131650588138227, 0.023449330780670352, 0.57676210288394281, 0.41250991392262581, 0.03037257261800741, 0.36969211568762017, 0.78391719704277651, 0.5509666070406023, 0.031454007797199091, 0.57775284734162557, 0.25430547218931687, 0.55940089321982966, 0.65715564989398234, 0.54770675975763627, 0.316364724110175, 0.64503961016463274, 0.5481003869085902, 0.017723602001872507, 0.623456736372792, 0.22485912496095894, 0.47384844253220948, 0.724829663495571], [0.77173231943064458, 0.22298250425335553, 0.083846009939146149, 0.370686136197253, 0.2870155577093656, 0.80506641413011981, 0.23614909750278357, 0.357746508096742, 0.671900394717986, 0.57802874947599181, 0.45888424787472637, 0.269944692174356, 0.62710351247536422, 0.18288163090708343, 0.037885364575030112, 0.11210455114786011, 0.26521916183394556, 0.63909904273888318, 0.94367156283075737, 0.39355044055251376, 0.12037351897023052, 0.76086173787746914, 0.31190810663154844, 0.24393761429563798, 0.023877341620540671, 0.45868683197901572, 0.47371873255981112, 0.88311051977332933, 0.46382711534504117, 0.76436318669355041]] [{"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}] +6 [[0.49869616208884693, 0.55144328654490093, 0.55377729558377009, 0.843431335623911, 0.23244546292038359, 0.911558929830448, 0.49086859023363205, 0.36226223621326314, 0.54370221709055011, 0.22534341652795797, 0.49156762889164018, 0.20495232148794884, 0.44319301722603754, 0.050935552057056244, 0.010168390659369475, 0.28971929477372882, 0.05303204383731619, 0.59729213564733585, 0.21790173337764662, 0.31804983072212933, 0.28812816272877129, 0.86135508638390645, 0.24114369881772757, 0.78738240307502283, 0.3430966972097963, 0.43550786859434976, 0.71991441868145023, 0.578146914192481, 0.84066355580305929, 0.19213832146796694], [0.85564343234407592, 0.435748629000409, 0.0063383984157834306, 0.97497754512817558, 0.51659120090499244, 0.10656011036556479, 0.7741723236468423, 0.63098527924958059, 0.89998097667726928, 0.10671112409077776, 0.29970499151180396, 0.201904270559508, 0.71180881535783547, 0.10344387282378553, 0.958744967286855, 0.86120739092823129, 0.81846409850816193, 0.67102632974758081, 0.544415919842615, 0.230012975029124, 0.84870629107123985, 0.74669437511917969, 0.039940966225225027, 0.03464551890238321, 0.28020868327962434, 0.45681855693158624, 0.040925852083370073, 0.67887002644545713, 0.13179281827990319, 0.10946911462007203], [0.62844117902184748, 0.19543041489716906, 0.50064240313976738, 0.0567785789413261, 0.25299261283515107, 0.21437348112399079, 0.83319873320254467, 0.81897839800357608, 0.97950749546020244, 0.039741716490676593, 0.99910444037652235, 0.33171221002814955, 0.26475492906044162, 0.66156987503619047, 0.94947573956155085, 0.42920516730123282, 0.29115567812893106, 0.54672158797487413, 0.98394791662766334, 0.70809530826607958, 0.76472710179465531, 0.34995390933581727, 0.42535278803041332, 0.73020921189786059, 0.99994799062786455, 0.26569954734617685, 0.80655068350784509, 0.11729130209325356, 0.060616121870874728, 0.60140740245691926], [0.51085372559859588, 0.46667209873104032, 0.70331172159417754, 0.29488454810879206, 0.17564138571706123, 0.17408768639372774, 0.87886797401546535, 0.52093346757057268, 0.0681829944815282, 0.18833917156511171, 0.58728964296381669, 0.23668050750855896, 0.073346429920357226, 0.44220738637074974, 0.95284911743829759, 0.50034260806963482, 0.89478567882987881, 0.20501463121214203, 0.48862910025973949, 0.66478168515654512, 0.56436461400100757, 0.88445182996105143, 0.6535694625921481, 0.095046443999420416, 0.37898809690941915, 0.86876624873941677, 0.97011549765063565, 0.88820417071534274, 0.93967534054963653, 0.84513120235817052], [0.16685315259997857, 0.18411578445530574, 0.054625259357372391, 0.65599564157952772, 0.038056853219922693, 0.047409459924232289, 0.886186714706075, 0.1624388351530629, 0.97086334236479177, 0.57375913884478214, 0.87106129432901058, 0.77203061242854454, 0.38082538835436375, 0.82767681722041186, 0.62372997920765516, 0.25842564324427131, 0.61788712739857266, 0.45437767422980679, 0.2450092718991036, 0.25089571181634274, 0.25736250631713464, 0.32488014346213812, 0.20080629618809309, 0.30423085422075191, 0.074287180463463054, 0.37135327044051814, 0.2105566253895309, 0.77291726346425382, 0.41455333998152433, 0.84882620347317028], [0.390492276426594, 0.80822533825039877, 0.12252586808206589, 0.61972576213254782, 0.23380220710688782, 0.6728190952704971, 0.66237735044996815, 0.69930979035333363, 0.87791033264656027, 0.2247237608256264, 0.579168680681511, 0.22291535637732629, 0.50851172005378142, 0.86096523739030018, 0.98574085106079035, 0.20004268872517028, 0.68067039912456417, 0.43092199305891676, 0.3327726515706303, 0.905647022762769, 0.79919710861223037, 0.63040589730639363, 0.32189806516722752, 0.52561234913762434, 0.99368890794445142, 0.16971568150473026, 0.45556641225035943, 0.7212324944164521, 0.14802680622505726, 0.57238281689459436]] [{"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}, {"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}, {"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}, {"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}, {"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}] +7 [[0.3885403004999336, 0.21153443164602725, 0.774279829955582, 0.38300482760961596, 0.43857021473500191, 0.88092208170327913, 0.14180047588093558, 0.71370370179496323, 0.83883388286883254, 0.52333036388707044, 0.64084484459061042, 0.54292690901596208, 0.2899485418945652, 0.24037804499810822, 0.76035846082846215, 0.75291951810081237, 0.15291184942847336, 0.24087319912159233, 0.95160123771273963, 0.58102582686639259, 0.7747184978237498, 0.2966266705076781, 0.55845907896814639, 0.58726020522682, 0.25207289853939252, 0.72422930224614035, 0.44123760571351778, 0.75247122800761068, 0.22624602224686752, 0.36596801119786526], [0.30544365146991681, 0.36826235845698541, 0.70935099628027432, 0.30184474348664203, 0.41991083511232219, 0.52924675607873162, 0.13078875830102454, 0.31470157297007451, 0.36272765763912618, 0.71887748269931828, 0.76234143681609212, 0.4517442455622841, 0.66877785029625469, 0.192008809730144, 0.43136803972705184, 0.378295220765292, 0.76930435066206038, 0.60050793767314969, 0.96378306578507045, 0.60339685076881711, 0.41176004645550046, 0.582651073788503, 0.11197562862643529, 0.071923934156867264, 0.827309320596849, 0.84270138933411942, 0.81755328766189128, 0.064154199247474009, 0.27369710783399281, 0.58398063733426475], [0.534770865479414, 0.82128431301143578, 0.22280240111846039, 0.026910380578020532, 0.99773175162295968, 0.82416787967510252, 0.38504609557665015, 0.1429124849589436, 0.25292505913528374, 0.19259130376680755, 0.96692318614130179, 0.66572305684468613, 0.99838925491476416, 0.24369203421434127, 0.043282304127351789, 0.90358791439818331, 0.27176887532762373, 0.85066321826958913, 0.692576083441532, 0.8689711902765217, 0.311757870009856, 0.87667741818911982, 0.59115788221762566, 0.76682388287465841, 0.98089942476561043, 0.62400143376497863, 0.091801789800795786, 0.93812203724176835, 0.48243888670554946, 0.93456325278650454], [0.421974739751959, 0.94229900671025213, 0.22990290621168552, 0.30577100244369138, 0.52899057090110269, 0.58036018032768011, 0.63530431467954307, 0.14417316810771763, 0.27202784838841287, 0.30988253853869285, 0.27034921502410036, 0.069588268565814126, 0.1150946104721724, 0.68376540532596208, 0.32474870902091857, 0.0036211203839363604, 0.21037512184432305, 0.15651291839602377, 0.53671493136814585, 0.62902201213918874, 0.777994640557613, 0.84499308469062, 0.80294723561407366, 0.69013419907788853, 0.98609031900189914, 0.60503158972073956, 0.488890673891497, 0.51971413521573562, 0.11417793113674113, 0.0908633787608849], [0.054776403617915026, 0.12369787241072328, 0.64315501428246913, 0.30137883300315471, 0.9889290915064517, 0.1330262384199653, 0.0568835289712436, 0.48526509293001729, 0.19705358308354548, 0.94043037096833837, 0.56314599640779583, 0.17655765829312642, 0.93468997826339018, 0.38011830493876342, 0.64587087768975493, 0.56278107574735814, 0.089751608292902185, 0.3795923167201859, 0.3458744156886957, 0.3497718760063474, 0.12303859769372094, 0.11536586193844067, 0.43396828267425513, 0.78044073536410818, 0.25684480092841544, 0.3381378873228047, 0.18813267588093652, 0.48914493113071211, 0.56525338123167468, 0.69802163547245621]] [{"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}] +8 [[0.23095541823576282, 0.98566778149208711, 0.27275395790578771, 0.75873731923951637, 0.10028729285478444, 0.2559464736048801, 0.28772272775550967, 0.56837815230575883, 0.051161029724324347, 0.13892752805400266, 0.51868974612042618, 0.69435764187724835, 0.883427089389327, 0.69140808027836886, 0.60124850508041028, 0.68588072061700966, 0.91656529819741051, 0.30389467847111196, 0.26237596645547145, 0.994322553771555, 0.10300189155805217, 0.87802175788641978, 0.82823553626372082, 0.4557048288135277, 0.5154890852913977, 0.88063429608427857, 0.041970919105270288, 0.89887521353447475, 0.83260111999349462, 0.37832495444470993], [0.79259573136394, 0.82351192199663692, 0.90831490279849159, 0.15719241890705504, 0.958622295711903, 0.85867720406469639, 0.50480713174259273, 0.81481455458044771, 0.20386284074678773, 0.76049550503284324, 0.79201421791406867, 0.27490611803013865, 0.22787495979701466, 0.036632984058031726, 0.27316516801087443, 0.51985778169966479, 0.669812147016539, 0.89560709006714079, 0.3824115700781312, 0.46232932894166257, 0.59197964606400177, 0.11055649630896514, 0.1188432658874603, 0.78429698284904192, 0.22818461423762693, 0.55995259174833079, 0.73059179393973583, 0.26043067252675067, 0.47200879579964339, 0.66183645224654342], [0.6433883335848436, 0.55240452234891935, 0.40142105043657261, 0.65351541580078243, 0.5398565125880973, 0.71171939682808227, 0.98868574932955233, 0.68525624962903209, 0.7389846669584994, 0.6608439394936273, 0.028312856378552853, 0.99436237109305858, 0.79963624773329067, 0.73595597670586066, 0.23083357981176, 0.0051250055876984613, 0.57762114409318654, 0.969293886968432, 0.3764755350068536, 0.060488911608155616, 0.8027643482331559, 0.16510321190628752, 0.4003497669710695, 0.63287586310982413, 0.61057341297389578, 0.98039065510194423, 0.28955115455991032, 0.28445953299197579, 0.36251888248365316, 0.16017445753448634], [0.7126130919160375, 0.35128785198563062, 0.51960725687424558, 0.79664601467012275, 0.37304608131521355, 0.31100009085510338, 0.29583900154307718, 0.98263505840897492, 0.18122885478367168, 0.060487484660962987, 0.311309281116925, 0.511996339832912, 0.47271028673348947, 0.87769785805392131, 0.489121762045185, 0.34301319661053487, 0.66601862499035114, 0.67088640527331878, 0.99595153052207686, 0.38563924473074773, 0.9493642692809533, 0.2289658273558185, 0.11300575930945467, 0.79937645467057961, 0.81316106541831246, 0.57821139897293561, 0.542295786317452, 0.39581420941202927, 0.50408015647336457, 0.22342733748484733], [0.80254792478197856, 0.92833048997637924, 0.76638976464472475, 0.0057828041350437287, 0.0497442933872575, 0.021183197650795127, 0.55762288585865194, 0.5448774767985145, 0.86919954764256291, 0.24119569815515407, 0.77815444717934368, 0.74919782082951236, 0.32924268898527209, 0.34746769017163615, 0.4380580081300941, 0.67697730533916378, 0.86098788216212119, 0.0680186409940039, 0.099651816192733889, 0.10857475070341394, 0.33434183163508657, 0.84699820289630656, 0.88043221225250012, 0.55183723775192506, 0.03730960181160714, 0.8820738409862019, 0.778632376349308, 0.043514698895639747, 0.13382448760755805, 0.42883288509242412], [0.59243281031957284, 0.032645992891135567, 0.57474226054089972, 0.78751294902881974, 0.24388918575336649, 0.83593660177573625, 0.36730316391640405, 0.6071008721579455, 0.88993255769637969, 0.64050086401134176, 0.57028022339456108, 0.142816790504665, 0.80533523479967428, 0.7959420887743538, 0.37519548480627807, 0.88522730829677443, 0.6563259046040828, 0.18559630396065652, 0.15840084690326972, 0.542707065305499, 0.44857516996876146, 0.79586658598986149, 0.44448309520044926, 0.70369582557789734, 0.12956306215143443, 0.21292640790922279, 0.30836788550791283, 0.63117375092189876, 0.5528781870183529, 0.66767238752755631], [0.86166073395158971, 0.54900483583986093, 0.27078798642076951, 0.422657818643983, 0.11817764368481531, 0.46827444026434994, 0.41145787866633987, 0.74884049713679246, 0.71428137840640171, 0.071473980726529107, 0.71848677537171035, 0.47532944907579111, 0.67155458925877232, 0.42336190921009043, 0.68296539183012028, 0.11191716726813439, 0.63250367844449629, 0.30955693985374222, 0.58205229159767258, 0.84203497329535881, 0.77440249146474527, 0.81739615299087531, 0.15310250653216806, 0.26287142402579955, 0.91204839361554113, 0.36831674122714, 0.57867193038148024, 0.60974447337349325, 0.13711740436411402, 0.786997238570247], [0.43526023902801736, 0.34342003412871569, 0.709969522543866, 0.14972012408805779, 0.26744395120440867, 0.40192158812288914, 0.81916870073738113, 0.092894951355157351, 0.48908987183916475, 0.12247583727700939, 0.25464677116931755, 0.9824067183281503, 0.38462946351831617, 0.82459236475241759, 0.0023543354981858666, 0.84543181435852777, 0.085912095203463212, 0.31187592238899364, 0.26140533956619683, 0.248589747111841, 0.533945656298537, 0.098021720046919447, 0.409136923730135, 0.3697755103116549, 0.59987106576796123, 0.99154589148043515, 0.99516950254571646, 0.3557443968268641, 0.73580687789945753, 0.78415449431440887], [0.9324663430047323, 0.6834970985785358, 0.75373904468502118, 0.17925638417736622, 0.67155609044367992, 0.6507398531968408, 0.589419699622833, 0.7699765353471153, 0.29349554193945915, 0.30917102011811648, 0.19827844328841093, 0.83012803957899606, 0.13781414413314719, 0.12407146805112124, 0.051638249145798953, 0.91695268387956308, 0.79359571913970606, 0.3946536176226626, 0.054067080071233842, 0.80104140373811217, 0.025017081631783511, 0.001849480268928283, 0.086486029175022461, 0.036697582077854118, 0.47707695098657865, 0.3636958586044543, 0.79469300775581486, 0.5651077583098516, 0.81291338957333714, 0.57319998798323313]] [{"5FkFp6-20TVdNJ-z6X-RLzp":83852, "75hGtv-oBTwA4M-2Ct-ZSwW":41293, "A5fAy5-5cSomjQ-CHk-YEd6":54764, "KvreiC-sjRjT9Q-Atq-Htcf":91172, "M1Fg2P-0UAY9Nw-QIp-hdFC":10236, "OEklRm-685Yl2D-j6n-dr1D":21189, "SO06ll-esmpimC-Qpg-pmIP":44277, "T6YWXQ-bEBE0TZ-hN8-eyaz":7230, "Wb4FT1-g9iLWg8-IdK-vRFv":39236, "xyXBbF-WwvYkze-lt2-D8nn":59480}, {"5FkFp6-20TVdNJ-z6X-RLzp":83852, "75hGtv-oBTwA4M-2Ct-ZSwW":41293, "A5fAy5-5cSomjQ-CHk-YEd6":54764, "KvreiC-sjRjT9Q-Atq-Htcf":91172, "M1Fg2P-0UAY9Nw-QIp-hdFC":10236, "OEklRm-685Yl2D-j6n-dr1D":21189, "SO06ll-esmpimC-Qpg-pmIP":44277, "T6YWXQ-bEBE0TZ-hN8-eyaz":7230, "Wb4FT1-g9iLWg8-IdK-vRFv":39236, "xyXBbF-WwvYkze-lt2-D8nn":59480}] +9 [[0.91932272657574032, 0.59023435137531011, 0.30619619687349853, 0.28849927487716132, 0.45352772432874588, 0.24655194318598372, 0.95006171941862294, 0.52677970703670773, 0.19017305345088664, 0.94864324630010666, 0.79828419865008893, 0.72078967469593136, 0.067926954216678492, 0.57003409487722867, 0.77364650920928346, 0.463415345503996, 0.24460360017602911, 0.2250260541917487, 0.23004676483632003, 0.45501787461780463, 0.78118117224476591, 0.61428816933442332, 0.891863056142934, 0.34818051922897408, 0.83063123556143492, 0.69252834123368556, 0.13560817395812519, 0.79510510532443068, 0.88664348168004681, 0.95771328504413045], [0.69874373818096969, 0.874567736968871, 0.55846320012735018, 0.83025145639280284, 0.139780704983406, 0.69741499853336264, 0.1554410984712038, 0.91574874698169118, 0.019301467758550017, 0.2679012119863925, 0.0885282061723287, 0.42639134795440126, 0.82329626889573948, 0.61273838930896174, 0.72583123352064816, 0.67370665435652555, 0.1082145774737725, 0.78055039896280531, 0.87547697087200227, 0.3393974583426943, 0.21138991095574144, 0.56149331836865812, 0.511416421370049, 0.18114005195023197, 0.14100431674477332, 0.68012434569093427, 0.70643304457064726, 0.9366830832581251, 0.93651666514195364, 0.774171710754161], [0.99150129181255653, 0.45227795333678944, 0.87348642788811037, 0.68767853149540392, 0.52177554521125324, 0.4804180335451349, 0.37598299266867341, 0.4790639427416018, 0.49687144356575608, 0.16705861270294253, 0.39527770755294145, 0.072964854773791443, 0.25673477282515633, 0.9799350677780716, 0.31527404020343086, 0.669292303973624, 0.3542761557482802, 0.71729206808426538, 0.15313917842654445, 0.86375650866124642, 0.6920820691577857, 0.2802815672782194, 0.93785779321934271, 0.93118503607398206, 0.50289585706443607, 0.092483474296017221, 0.66993682929975817, 0.87627886169765956, 0.55219527633119492, 0.40644716904099942], [0.19224779968930084, 0.66602167905617926, 0.44607563395436134, 0.24984815151673778, 0.63500124293408378, 0.52377731509013725, 0.33878074983799389, 0.43279913376012558, 0.83710512977529183, 0.28718153344657837, 0.65160630111109141, 0.86304996466124273, 0.29166755410676704, 0.61025387184805036, 0.63853357033999614, 0.39725652950621593, 0.71516140318200072, 0.22472599178525743, 0.76383383133431382, 0.594193907346904, 0.2977766623948277, 0.56640324782238549, 0.89301678290632758, 0.029734136528713484, 0.056743750993667463, 0.81944627175356011, 0.24087892889251306, 0.28360455774393734, 0.48491923893190436, 0.83478001405750635], [0.8690412858125699, 0.3143879807530624, 0.86368302957329113, 0.43840775907703211, 0.28024746140689605, 0.70192113855255378, 0.19220362306223537, 0.36122792086201805, 0.31782739870135956, 0.15519312577051192, 0.67438666743321074, 0.39402211247307994, 0.14611342267009164, 0.75904106446453179, 0.10032882775067842, 0.39960266203884565, 0.5612293671475681, 0.66438538502333488, 0.26085739560093135, 0.498678614463528, 0.26155540218333473, 0.31444253798871424, 0.74174411475153812, 0.086819415945733058, 0.99330425811157685, 0.8432979788636038, 0.058438698496842467, 0.33696559994644326, 0.679028017758292, 0.98341809934972935], [0.11267611252594467, 0.49525626536408163, 0.735506108443002, 0.67512449271450736, 0.37906900325749715, 0.58751364751307922, 0.79234941362711075, 0.5837497881969862, 0.06280490065765032, 0.11954482539675393, 0.9337870714910248, 0.89913927277105921, 0.70882509434381191, 0.97649162617463181, 0.40348607924116964, 0.11735774087685802, 0.38837033923079056, 0.0047794421430829015, 0.43588629732381345, 0.985906816634858, 0.6405364360917295, 0.69478664918530431, 0.50741632217011312, 0.43145284375220139, 0.80448973164335125, 0.97317591139210147, 0.83708835866841724, 0.08301084083433441, 0.24225164255334231, 0.36089954171643812], [0.40835362792990515, 0.95372543768547779, 0.22246810677046691, 0.65621131105023134, 0.13273177180910389, 0.95083499240707581, 0.91233802477738135, 0.86158683681530324, 0.030611128392776688, 0.14610047810278748, 0.00788269323903179, 0.28927602320431578, 0.46574965552351844, 0.15792571513231268, 0.021995939164806355, 0.65228043701672589, 0.53738837435489828, 0.57159124261263161, 0.1452222682307216, 0.47586909995198079, 0.70752308474521752, 0.055469200337252889, 0.8959687105862072, 0.34931670854978925, 0.62076174567336073, 0.71878210235542073, 0.48486308023477453, 0.37836844378098056, 0.8603288318546678, 0.30041317873450657]] [{"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}] +10 [[0.68383567150442692, 0.46201676716472917, 0.69037292452019283, 0.99961567844927268, 0.16449502791842407, 0.27555905654926693, 0.15235547393011295, 0.11476967209432909, 0.068832990208895661, 0.8439681913099778, 0.20260637712199403, 0.55626934703922992, 0.089663151057897017, 0.095733193192900523, 0.660586309210131, 0.0042917259839827526, 0.71207824643690554, 0.9046762692928767, 0.41447050633200266, 0.628729432515436, 0.86259477503332715, 0.248023107143737, 0.36614670864997423, 0.6958735657364169, 0.40696963598869185, 0.075121798826113251, 0.25456311913275609, 0.97853411645325172, 0.80824255013008073, 0.17810058673443174], [0.64603158390272153, 0.10121967713471614, 0.71847250735329415, 0.356731165379058, 0.98262937099940317, 0.71860077452209492, 0.27079993071613162, 0.95199119019131861, 0.21363272575388537, 0.47333192227387866, 0.36288119932030871, 0.3862957585564788, 0.19995393666925321, 0.0084096700205120722, 0.27840539364901473, 0.69309446014115528, 0.77784304192133591, 0.9195949516848152, 0.58031752441033624, 0.14450714462928538, 0.095654053215553425, 0.42503007056058273, 0.32753681251604805, 0.99289804875207877, 0.59626499696310642, 0.702031928197267, 0.3163722522576311, 0.030764346091181172, 0.80777977186387029, 0.29514475600195655], [0.22850181471275044, 0.14304212859703569, 0.92642918281217113, 0.73488706149580829, 0.27929612961767969, 0.27690910600322949, 0.028127475592679252, 0.52322939610059083, 0.20343075675237965, 0.28638979750098548, 0.62892945882205342, 0.93969077059497474, 0.1891462472408324, 0.922545823276339, 0.22967311071524921, 0.70104298203932425, 0.67817819696558168, 0.83476038145136355, 0.42638073140338717, 0.36796442767309889, 0.78529438714226529, 0.82460214841238455, 0.14553130124313263, 0.53358546993039135, 0.45953961274178345, 0.75098142578949034, 0.57210590899283487, 0.25172006090664079, 0.40870334677390863, 0.4237864713685503], [0.37834648875454491, 0.86667896201804251, 0.23918893599751745, 0.54291588388164025, 0.93245800557693859, 0.39191208150330159, 0.22537755325365494, 0.24059874555103022, 0.26902258694290893, 0.85132881463937349, 0.61574381048550231, 0.067120569158939269, 0.73668371244547171, 0.54499668824719127, 0.40739434413709208, 0.323550752942801, 0.40369718181013936, 0.0300205590872743, 0.0763644352023094, 0.48979223816377349, 0.4982348807834418, 0.71775553046660068, 0.83792201182023862, 0.12225208300824852, 0.38786659531658962, 0.380109270302761, 0.62929235361605818, 0.0026574673494252909, 0.86143373558741487, 0.53053976702538586]] [{"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}] +11 [[0.9486624071469375, 0.45111107550701379, 0.34166552478450252, 0.075296801657753654, 0.91907102480777458, 0.96539452643299128, 0.793899676066837, 0.33109932125166563, 0.34468953382531931, 0.087876614564969757, 0.76920181976769408, 0.90334077262058576, 0.77523257013270075, 0.84896407360246118, 0.44543268511641465, 0.5027209394872445, 0.31382080405006485, 0.43233751686839339, 0.72352293042907212, 0.86615406904184356, 0.64213325871723559, 0.14347875353396433, 0.79003470918577223, 0.92109555206168237, 0.38823878701923176, 0.61779623621104063, 0.20912432350075161, 0.8030685046469056, 0.27231757083916741, 0.10614793464422045], [0.27843292774062856, 0.24241184010266748, 0.22817946916718856, 0.25856838581985997, 0.099253921366206921, 0.49740485134206758, 0.61743072325401782, 0.81626474153309325, 0.18339453799005578, 0.57830521794904965, 0.72833649062313122, 0.017454202729087487, 0.4474707850911388, 0.98047073266177376, 0.98287119843405835, 0.32437317492996, 0.129944010995647, 0.52461788295686451, 0.66314490966464945, 0.28679647294840771, 0.507028621782214, 0.18840285874384688, 0.51233838538762455, 0.97166217101277752, 0.28067041343275978, 0.0058936299982919049, 0.26547552170713218, 0.40890621824047146, 0.1651149947341769, 0.21013473321597254], [0.021806464770294354, 0.93036806669055139, 0.50815668860594154, 0.79888869700990017, 0.3832710313159956, 0.11163709142047751, 0.74897816163240882, 0.61477120452056477, 0.45245579393407087, 0.26596650903152874, 0.13656149764490988, 0.84838015009039969, 0.10865552846561521, 0.084890220577363618, 0.41097192008253358, 0.59665080579675456, 0.4620930810883267, 0.071935223273759119, 0.084385164305907034, 0.73855281373393689, 0.024048276959301562, 0.87383023306247787, 0.54955049749285434, 0.92637747191346509, 0.92459158186691193, 0.47786892346950316, 0.49045109171460244, 0.93324591942302337, 0.45981299056272151, 0.51459416239237066], [0.97324418495618648, 0.94610273423450775, 0.35688261492576034, 0.58172726611007042, 0.15513359097833446, 0.64834111146648421, 0.40797434237258068, 0.58827406576912755, 0.76464732747377606, 0.097339616631213488, 0.54302944360088123, 0.60699764487262964, 0.3788637507741861, 0.4695726182280695, 0.98653224967362074, 0.61895504791430322, 0.05144963093819177, 0.71958953448920937, 0.87491512572487817, 0.8355536150768631, 0.029944086675009252, 0.079041175477049919, 0.59890588097142239, 0.99158955617277866, 0.14154231626149105, 0.10806345747775969, 0.17907522216968763, 0.11699039581571935, 0.92370388989253638, 0.70371271117211587], [0.1830380541139367, 0.815489337108797, 0.72923011831845175, 0.068778466464925136, 0.049206839553808668, 0.005218722635876416, 0.84502581303401769, 0.14918322194770783, 0.19469163691311109, 0.40921649324520704, 0.12958054833246369, 0.45931121121612006, 0.95388526276016683, 0.82309819325836364, 0.79627553059513567, 0.531990917669325, 0.21946914817075269, 0.33265444157230306, 0.74794163344070452, 0.45348918319248566, 0.63901913861282122, 0.20589143136901955, 0.30266539176524576, 0.073757086678679529, 0.35808667133109751, 0.14204060735806057, 0.24165927639471507, 0.51830134733955935, 0.59457602515220376, 0.92432035253026523], [0.85708335937531221, 0.57043678044698587, 0.77943336062061308, 0.29374154312744716, 0.081025650298674257, 0.34130924834833665, 0.60510879847052146, 0.038005611374709813, 0.83641860705000326, 0.399960539289254, 0.085948002074791252, 0.87255212007809579, 0.46648980859439093, 0.19683975529959363, 0.65361499231733067, 0.68007073019890651, 0.471414941734445, 0.18154863783748743, 0.38584358893749404, 0.070299787273821246, 0.80916167932615424, 0.48924092525201646, 0.698259525184013, 0.023691995568354773, 0.702083237392583, 0.44997878135301017, 0.21671132167854434, 0.590535359618753, 0.23164082506013417, 0.46148200812397533], [0.091492748516129652, 0.23137527287477988, 0.0007042929168112888, 0.95808876882872662, 0.71503534229241339, 0.40509142681111432, 0.79620111513973379, 0.30737903617658424, 0.66109462093967075, 0.43774135444800855, 0.49445323299059163, 0.47079525512721609, 0.67614915863827807, 0.53828971761954958, 0.41736272890448767, 0.17572702789846673, 0.72267639825779473, 0.066600700590826567, 0.96967667867920715, 0.16390983753282073, 0.39916544846391255, 0.804578548933932, 0.66588237578358622, 0.80462030175393184, 0.54317111349065161, 0.033098943269978864, 0.58985412686534233, 0.991033740524661, 0.46141007221381747, 0.67538188019806966], [0.92119347126658568, 0.774241937045562, 0.024844110055014079, 0.63171460042756566, 0.40927454923779116, 0.6318125104132527, 0.81689188740650209, 0.70094079720243818, 0.57932713231064392, 0.61640375069304609, 0.62935483530108449, 0.1608899906663831, 0.014459657400569315, 0.88483156896456538, 0.86333929053886949, 0.16292922151583822, 0.27334542571303588, 0.26911810285074123, 0.81140107554681806, 0.58507665471243175, 0.8753649267671193, 0.61843441763380536, 0.37233507185204229, 0.88362448326959186, 0.31331802358093097, 0.61448249038386549, 0.481477040992803, 0.78497669809123494, 0.23838971404817155, 0.64274419997173293], [0.42708050146599752, 0.36741459082608019, 0.80287363237196241, 0.088158576401929922, 0.26727651891094217, 0.19676113045319155, 0.64322780686711489, 0.11617030206708978, 0.76232020986560445, 0.70042293190278315, 0.13935738689047505, 0.5987266074187324, 0.593272108609207, 0.89104637011544741, 0.39558943483763209, 0.34968101896290793, 0.1509588632018225, 0.8840321577272362, 0.85462528514033487, 0.24519662046836177, 0.84800218124879922, 0.39241063745601867, 0.57782608808803426, 0.12212345282021764, 0.73190102502729881, 0.52808797734348611, 0.0035074142127339281, 0.57095753330756727, 0.15859652586192896, 0.79023074262598059], [0.42948962705246807, 0.86987231896743833, 0.42364539119463807, 0.079564665799524037, 0.55697042471168445, 0.34244563179264786, 0.54207189996345084, 0.94162120728229248, 0.17985645581872256, 0.36408831151890619, 0.81816464797555122, 0.012472901577568418, 0.50701144347988991, 0.090813391099027085, 0.45292912410740016, 0.81725662068784144, 0.66444845891964588, 0.73886771215140423, 0.19960240089947823, 0.65479187855770815, 0.64031385977594613, 0.90488016722340858, 0.9965823948892869, 0.29085849187844426, 0.30758171972706039, 0.98356442271337219, 0.5823178953786684, 0.57820638983316508, 0.1876966456743363, 0.21364579229422442]] [{"dqI4P2-eGHUQMD-iUg-K3lO":28601}, {"dqI4P2-eGHUQMD-iUg-K3lO":28601}, {"dqI4P2-eGHUQMD-iUg-K3lO":28601}, {"dqI4P2-eGHUQMD-iUg-K3lO":28601}] +12 [[0.058173649168726693, 0.551773880729322, 0.57321059200671853, 0.26016350610151662, 0.1574344683181309, 0.26284302217925104, 0.044477789530217415, 0.28580780548348328, 0.30861084862821653, 0.5164106170109144, 0.93623389406263968, 0.36984161172004326, 0.72020196597094288, 0.44918890774554188, 0.13904099897852906, 0.083955447505741132, 0.41689989968855878, 0.87366108406305443, 0.22879438213432668, 0.7782130160230053, 0.13749845911267267, 0.39231303237113857, 0.58211536755929838, 0.29013593675986893, 0.7333292781314108, 0.41733804468965241, 0.57663713834505836, 0.89423759090005772, 0.36315418582856884, 0.74647945853563924], [0.89916372211944262, 0.733002916741309, 0.74235700341949939, 0.84665312120824643, 0.08722063841225236, 0.23691378833125121, 0.19744189459956407, 0.27921831710125256, 0.56330516328866043, 0.58896950729609243, 0.30125896958372245, 0.76424175446606557, 0.58618322151541435, 0.1182553163727319, 0.42004303910816343, 0.22582649437486824, 0.739868679444697, 0.617461655190474, 0.59026418037096151, 0.786553786294658, 0.80846035563420238, 0.97086009812772645, 0.76145370186620043, 0.0863122908059678, 0.14380066243496348, 0.15209189467401918, 0.59058303416985114, 0.48926623979437611, 0.92295906459730626, 0.6052279437391096], [0.65423668461699869, 0.71244008368193, 0.01383277511423775, 0.25790973386824334, 0.15673500167214272, 0.16037402619911556, 0.519695663290712, 0.93198272841520613, 0.95552626679441033, 0.7825314322389294, 0.45731619234979604, 0.71488502064864878, 0.9995817449293608, 0.583615398581763, 0.43830323432127638, 0.93415587160357361, 0.08219142179075678, 0.011678754407532743, 0.66962883691705744, 0.737780302686587, 0.14647213252109725, 0.32488790350249075, 0.066814955331279235, 0.92468691777499934, 0.25006407427253063, 0.69502779754435284, 0.7820612966609618, 0.59256882589143178, 0.69490531125335209, 0.24431153910164227], [0.98962944948955811, 0.374297738260975, 0.75863858317665533, 0.43178526333686307, 0.34208832001764666, 0.51955405925962161, 0.92611808793758921, 0.20659480368412264, 0.11356466323566272, 0.84489651968533153, 0.51264857914074491, 0.1276077421775077, 0.67993149431525679, 0.62334708489725377, 0.0950028189218185, 0.212360663672004, 0.65090628734044331, 0.813667752959494, 0.22192754216583721, 0.62302830974362988, 0.66531999649787132, 0.63985827021400821, 0.46276229189237172, 0.42308775989381009, 0.58835705399615579, 0.42300450014074809, 0.970959374924912, 0.54557687402412691, 0.6083551175489299, 0.74045930028476559], [0.38870741763807348, 0.80232399284589917, 0.52079320167097831, 0.44143924998052353, 0.099524962634896186, 0.23325072238690336, 0.30838508517969343, 0.78347183091450279, 0.54760447706579385, 0.34582811136821345, 0.97873432868231736, 0.970729557505703, 0.58585154860017485, 0.82699130477396665, 0.50479651935847158, 0.6731390713708304, 0.58897779331315092, 0.1232957971693035, 0.62843277076160053, 0.68425457729428307, 0.3736702979471177, 0.87982375477229968, 0.23362844174912478, 0.73623662437497994, 0.804774224070059, 0.60432651784340641, 0.97348902903874135, 0.6318791253726519, 0.33458933314305928, 0.080315072888647254], [0.84391547910837672, 0.27647854213831713, 0.62179582824263158, 0.99128736514300453, 0.43689913784981449, 0.89644104411968306, 0.80718756166517713, 0.93800766004086467, 0.67766620406218658, 0.71687520170667207, 0.9853067369926839, 0.531730024489774, 0.9524405956536709, 0.63701854235781341, 0.9042360951249655, 0.52848390952712854, 0.66738296273807463, 0.99019908505943177, 0.9502206354404098, 0.033017308554187053, 0.4443027708127768, 0.304133352071531, 0.029453407228199113, 0.64571862744204422, 0.35502955735674491, 0.087714597839108843, 0.046275041525670635, 0.051815453040739423, 0.82386827886413416, 0.062046455889823937], [0.29520029226860212, 0.95937516116430466, 0.086352902291897737, 0.92632845174258649, 0.23359872126024217, 0.48110816792481537, 0.38255147400356415, 0.70330207031082737, 0.78252127072401745, 0.024701760015774288, 0.38446480563474095, 0.94431655818523919, 0.51507825550310771, 0.50738915636822057, 0.70960023103424807, 0.59067657174999366, 0.83493804916787651, 0.24268408496955607, 0.75659047905983012, 0.085783935917387866, 0.52193040296494608, 0.16068901260300694, 0.51376754048241569, 0.41168847507311368, 0.82740203124205147, 0.11811053306794683, 0.43788960765378138, 0.084013766468802542, 0.63888610321054851, 0.93009283103101437], [0.97775883236075445, 0.780825533297329, 0.10181164336608284, 0.41320624127377537, 0.82625651544826206, 0.2038618244292294, 0.48539259300195969, 0.16740471350023078, 0.52436139482023869, 0.97464564181616609, 0.010500704519062376, 0.044492308756156951, 0.61200559359492812, 0.85335258780401124, 0.81923987463184866, 0.25161572307223445, 0.12316236196449593, 0.33490717624747812, 0.25910452019057906, 0.82052625879635155, 0.85634814694410577, 0.72463326290335517, 0.17901770411610129, 0.73683103165756525, 0.51676115277967, 0.1248857527341305, 0.99020943064810729, 0.36818021225953612, 0.52256713143868794, 0.3744419470479331]] [{"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}] +13 [[0.089452985160848875, 0.24145592288378026, 0.91129120931446017, 0.13296414413215685, 0.0741821651267025, 0.23983795296183663, 0.83049738874378731, 0.29733913780505616, 0.62421994936966962, 0.37564059775751446, 0.10746463385204752, 0.23260767634931867, 0.550333181036779, 0.62262675190154826, 0.58639853676548848, 0.27594914860221353, 0.8491383732091321, 0.97641469307579987, 0.26025381674438552, 0.1320540318911203, 0.80733606326079155, 0.1765676030816814, 0.077226223567426366, 0.11180611164823462, 0.527222309707794, 0.13201159962834974, 0.11671816080762465, 0.017728349116398645, 0.077731224864236226, 0.3870922745254628], [0.068035503891654914, 0.808913732973737, 0.97233332550918716, 0.65796406593262213, 0.14526949290684088, 0.69866944714792745, 0.82843409907137278, 0.067638423915151069, 0.9282621975187012, 0.26486120764347243, 0.87248841538341915, 0.21290017483542356, 0.13224947388898367, 0.48595207306522492, 0.42086553855363018, 0.60743414336735091, 0.97772077137797453, 0.35498786210128597, 0.58857095049686092, 0.36288683032736, 0.65723054047089846, 0.86928930791198555, 0.2562507999307172, 0.1932206955624638, 0.37734086594927885, 0.7453686034090381, 0.76047539920203078, 0.95760186446429529, 0.59157101446773219, 0.16659626469536482], [0.86311825644054885, 0.99319418045605012, 0.33417390786608225, 0.23540137002400663, 0.30271776982914023, 0.43161590781759096, 0.014755972146636931, 0.57229670701794766, 0.63402894075590788, 0.541447993112259, 0.64020101593791612, 0.68077787189153771, 0.12154228327341365, 0.894065483524574, 0.19305496357649654, 0.280153192528405, 0.16856844088356304, 0.81119683423884281, 0.8144732923252721, 0.79761895605849364, 0.13555608329861424, 0.11999834170113721, 0.51502785438362519, 0.60640289882289367, 0.429632169167375, 0.71506379868778458, 0.36811611586629933, 0.3985557291220756, 0.43192294683630672, 0.80667257624430433], [0.52841009585100163, 0.2889794454105592, 0.17003531126427662, 0.020711653678343711, 0.96172822381486756, 0.86153772056550437, 0.640147392419355, 0.7080904141267963, 0.47290862871576045, 0.38860266820952272, 0.33647885509426712, 0.92009958613738885, 0.47470717663755735, 0.88214931515268424, 0.85548027568905627, 0.51104828626665422, 0.95343968934960244, 0.46751711467212509, 0.40790267135718083, 0.74064677678298318, 0.25603232740526682, 0.8102158501920429, 0.69397727756498884, 0.56598410732685933, 0.64833980563323268, 0.97166088072968471, 0.875257157376079, 0.53658389408157547, 0.076036570052034014, 0.12148041957715749], [0.31124150067684797, 0.83590332137394174, 0.1970809904391656, 0.99204578862354, 0.31972460212453935, 0.54877384335302026, 0.54869809979547113, 0.82513271335913407, 0.4668813337781853, 0.67666470780574151, 0.66202193412406807, 0.83113028094347308, 0.79642137441576089, 0.77128300571481312, 0.043983422738506861, 0.85797125513331607, 0.96671489659887289, 0.31190731821280282, 0.2087256749537576, 0.049966141698995381, 0.48061247209036706, 0.11298416057873084, 0.35977288519973816, 0.76598009032380687, 0.0647524707253706, 0.11589981425504337, 0.89937576492886318, 0.92190060600910562, 0.78346949692792323, 0.89247252755694129], [0.6729000417484704, 0.22410129509896448, 0.65698769997439288, 0.067593346250181763, 0.74382162600110413, 0.46861460415755185, 0.39440315353479516, 0.65457592032509448, 0.53883590152375227, 0.19498463467283267, 0.34902950974143732, 0.989159399017759, 0.40278467456393774, 0.504278917666046, 0.95700205963968377, 0.54674042594771843, 0.959549647484965, 0.77749537564721849, 0.72960522331596767, 0.43229366271885006, 0.18775615507669219, 0.36360920919929984, 0.33560843705823962, 0.74357286298375325, 0.789076652106435, 0.045761707809252261, 0.43196684699645449, 0.32556731798239746, 0.25211837246464508, 0.23960111535846207], [0.67956232919706072, 0.6578212307118686, 0.99632542141653246, 0.42954308551294162, 0.81523123296506561, 0.7461068601134887, 0.11802010769196847, 0.30672690806921121, 0.74214574137292388, 0.92607423143004786, 0.55465452186172359, 0.4367003554636909, 0.34768557766514741, 0.42888390382857389, 0.93353780929245667, 0.055374772060051813, 0.16125743339614018, 0.40746153265449636, 0.97775071211410625, 0.768549465322035, 0.024089672446446331, 0.87689601135655926, 0.10465356132179215, 0.51331200297139212, 0.65885017047581673, 0.68363566490784511, 0.77133968448018464, 0.12887953798917218, 0.39900571602069956, 0.18551993319670812], [0.1965813236587205, 0.27995331515198174, 0.36616116934649634, 0.79831673763396793, 0.32026051136954348, 0.27741681541790353, 0.068187500373316445, 0.94431835652634866, 0.24897252707527395, 0.54921574994780353, 0.731767597828022, 0.22468172574615042, 0.980435164228272, 0.12851568149465453, 0.6800596977173915, 0.82958175171499771, 0.89953218372728883, 0.54674115747316121, 0.719497693354742, 0.2988789803547095, 0.39526428235218791, 0.945857902757741, 0.80931887387125723, 0.75088600215596535, 0.96057710957541842, 0.057267466997572791, 0.25322924692719695, 0.60794948521280023, 0.3059803131155493, 0.93394146786090471], [0.27468723131730022, 0.66494772824701676, 0.34306712881524137, 0.9679304267433696, 0.11560663642059088, 0.070436369243278563, 0.3754898045273346, 0.77714023160250878, 0.73285055086105311, 0.42005795347825214, 0.47264075996912058, 0.0032765290387108958, 0.75770732868874358, 0.69796329516510514, 0.79523526384562149, 0.92711570717651548, 0.63061128299422486, 0.24437589679891059, 0.59218643998292775, 0.077768734565688, 0.72668953358672506, 0.27606355495712431, 0.69460860903130373, 0.68936184738718442, 0.84728847394751872, 0.74136351686951885, 0.59548080833203021, 0.941566017122033, 0.668451996406095, 0.45254091686079589]] [{"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}, {"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}, {"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}, {"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}, {"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}] +14 [[0.63221100020486043, 0.66419100187251068, 0.54223407509079147, 0.55614475388159912, 0.62567698285273765, 0.14408921908249051, 0.56632600009444878, 0.98746156764645143, 0.3248061233757219, 0.40763405487584814, 0.958919854781932, 0.88413941373312155, 0.86763180024487319, 0.060569617157883893, 0.15713202624220213, 0.18452066167354397, 0.58431245652517017, 0.94655573283297623, 0.22745176644958731, 0.4239152598710878, 0.15127906689835824, 0.63753249863689077, 0.874850393117698, 0.69835449664946836, 0.18242563808798706, 0.6441862750350752, 0.52200582984756139, 0.48505789848958369, 0.96123103448756142, 0.20787559715793502], [0.97942435183974741, 0.20489809225267197, 0.59680130710888124, 0.8410398345460991, 0.82196923738474537, 0.35843671074313821, 0.74717374745210341, 0.62002720673843326, 0.96328146253169766, 0.044644080282096654, 0.19482533021478765, 0.60244335516218339, 0.076804729483281986, 0.68020430457242875, 0.98217954164251975, 0.64053484820396189, 0.4506037084964003, 0.777346360008795, 0.51900758863901553, 0.29278695950197908, 0.58289035368090414, 0.7094310182273702, 0.058971082876968839, 0.49782860565431419, 0.32615076025601986, 0.49606224585597158, 0.20915532692515604, 0.16790152994652041, 0.66999037534339312, 0.77328112298770391], [0.68429332887915306, 0.88356476439363463, 0.3162928073857868, 0.27943912347086008, 0.34648279882084865, 0.42696998743638848, 0.045013901339934881, 0.3836366908530775, 0.681549712385942, 0.9779563391514231, 0.49490885825544351, 0.41115217041893937, 0.25481310023060844, 0.41537874705690103, 0.39071729002112232, 0.77907543942660684, 0.36185866812454248, 0.31549672751862756, 0.43680733538941863, 0.24539894149706287, 0.044028425548914396, 0.9199151029104512, 0.067006692785546829, 0.89610889974352093, 0.87368692810757709, 0.55748260803903915, 0.99132473989960879, 0.84735756116052063, 0.43092527054211072, 0.74771717766891688], [0.012496927234595723, 0.023587092609008931, 0.1912155365825613, 0.48774247434239959, 0.898758683710582, 0.78751444972047635, 0.62263064673696733, 0.030382539482552096, 0.1939829452450117, 0.644175813327713, 0.65159521024981593, 0.7979769024076947, 0.72229189004552707, 0.71876791602486645, 0.29773545050484229, 0.94464174165867276, 0.20148016301405836, 0.21681314997324119, 0.20124700253588712, 0.47911694979362418, 0.81806296170386172, 0.61521392325881541, 0.97030155098819126, 0.791003163458028, 0.83648984935162052, 0.90405107430982057, 0.76136466656851121, 0.8851067913753019, 0.31604042074127836, 0.8015201743131376], [0.27189847928804567, 0.62177507214604433, 0.7911459361316322, 0.49818855915111404, 0.37277493641051818, 0.78551710218288517, 0.3343563182720406, 0.15179492021452956, 0.93682635011443183, 0.859871590120638, 0.765899380121936, 0.050102605160703195, 0.030311994538840259, 0.97339497150622256, 0.89975538901308361, 0.0868565561096335, 0.525680861065447, 0.59980855430900726, 0.62339634324741777, 0.95468304228339362, 0.36686218866382725, 0.24475716433502259, 0.29681642926934559, 0.2488103710436762, 0.17241700818754735, 0.53229543756504016, 0.42620650677003546, 0.49677774958216614, 0.5972356607558067, 0.81613256645269772], [0.81860158240723835, 0.9818387171915387, 0.89006744493915, 0.68382989295445462, 0.45605773428652341, 0.11155431513212255, 0.38202064590028573, 0.49656013006448474, 0.14139039678019849, 0.99811071891280012, 0.22027199655023322, 0.4127008447901459, 0.5622333766756471, 0.863184459371285, 0.5215771560027973, 0.18044881937967039, 0.79615434760381065, 0.85771803315777351, 0.26177436865874637, 0.35280659044731288, 0.43678829233475958, 0.89575747452759313, 0.64032585811885179, 0.47770541854310622, 0.45942859088689325, 0.14154249102343097, 0.36304240575432067, 0.246694157195802, 0.6642400919043715, 0.86032889176325245]] [{"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}, {"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}, {"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}, {"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}, {"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}] +15 [[0.63885556051479053, 0.74535962205844652, 0.67486103922792462, 0.17793005475060286, 0.59375151527306469, 0.12581600279247906, 0.28116095067089286, 0.51338990857530142, 0.72721199812358517, 0.694769409584071, 0.98388721809677948, 0.61233900076897285, 0.81083138973737268, 0.3419476586429343, 0.28120160344629874, 0.38455139677956807, 0.63223240609624642, 0.842531471683803, 0.24496461148998405, 0.035380584800178805, 0.50866799829363474, 0.54379505313084409, 0.85556767354120733, 0.33144679801048915, 0.78137914106922657, 0.76596973992721984, 0.74081547643732781, 0.65923695212449118, 0.42172032790049663, 0.064211490243435421], [0.75699886622210344, 0.5220331221341511, 0.78338512592403753, 0.8471864082908469, 0.95674466603689856, 0.7973440734717665, 0.094634284994410067, 0.10267088701853488, 0.7509241367276358, 0.16002133411119368, 0.50024953587756893, 0.14169598743504008, 0.49084634877333633, 0.5654997150291452, 0.33500607141996419, 0.34014830466876944, 0.27215910754946981, 0.58310971747006446, 0.83877710773650582, 0.83037261622607617, 0.49332460006838774, 0.67116721790384959, 0.707954929625355, 0.97520415825326523, 0.58923256061975848, 0.19355345037687621, 0.71869185764408927, 0.24299150384167811, 0.40445899505003491, 0.091756371092565958], [0.1545361188196871, 0.79455330792326406, 0.87087389521336855, 0.64685196333805473, 0.92435712886100352, 0.69164585636065, 0.01074648426337399, 0.69266244221676165, 0.06602541964776687, 0.14140008948175298, 0.79919225400539762, 0.97433514465777638, 0.79713249413831155, 0.6023205560556194, 0.088750718562418451, 0.085844970450387414, 0.3961064984656586, 0.89482092686777848, 0.62407890261831622, 0.56542982112535423, 0.97539941187248147, 0.682903820629692, 0.15619020176355103, 0.27972719963480197, 0.64299249582788753, 0.82347542784335159, 0.80336398978236179, 0.18538754280166847, 0.63928633196377971, 0.11591811806659669], [0.0997964745527028, 0.92119759954517444, 0.78351264127833953, 0.92692737452561513, 0.62325886366158767, 0.45425522603966917, 0.96712496430355754, 0.01425157178570613, 0.699473566665951, 0.18049471838165354, 0.49119586255277015, 0.62785683041714513, 0.44361551105344876, 0.46286945264004964, 0.76066865168086784, 0.62817103491396409, 0.31828986868155063, 0.57975658348155834, 0.22038175170071661, 0.9284446191148652, 0.46253376602310403, 0.45165624004743921, 0.44610419495548748, 0.04189826953178, 0.27783392121341222, 0.3160158113694943, 0.98839108720272462, 0.89421037399870451, 0.14484836877038021, 0.72801699853182678], [0.90788843736623326, 0.3480302681438916, 0.68846514618416932, 0.8794219645166137, 0.937786296942425, 0.078585525839186987, 0.709026260520254, 0.24808230399604836, 0.86244072485364875, 0.53808513088051269, 0.75399078699985234, 0.017472025353283072, 0.010753755635567797, 0.75150233076753425, 0.31700597916796125, 0.53861456743250125, 0.72872272062930055, 0.5974546027349934, 0.82030059639437147, 0.34603042804882711, 0.31763878586328764, 0.045280268294567105, 0.86151029397823009, 0.88484081988798369, 0.2027364748039272, 0.54304467689273006, 0.68748995016995773, 0.15792003269267263, 0.031664831628613888, 0.38109118749346982], [0.49536293824136624, 0.95999702923594865, 0.9929723986700324, 0.39188543551277577, 0.95500329068021561, 0.22986505770747068, 0.89186781514453017, 0.78928403184380391, 0.2973412412873605, 0.32236501115288052, 0.42383802209942234, 0.29899125834540785, 0.41932861553483924, 0.678567258270969, 0.64627728733363, 0.98685100513957125, 0.71909552621132655, 0.45558323883937268, 0.86409913149721629, 0.35275295052671307, 0.79377474400929038, 0.16669680616147065, 0.017375256087800439, 0.32531583759536187, 0.70766678413879758, 0.27801293425758078, 0.18736418912457153, 0.76614268896544879, 0.26986200958326612, 0.0095788086263627337], [0.021864101602305785, 0.97232590279467757, 0.64813354353825148, 0.2969405740622757, 0.72164359397874822, 0.45201336101498291, 0.68985825265228562, 0.9833569291391302, 0.72677493221928158, 0.94022885206504458, 0.5209196018715877, 0.83339788752336719, 0.39501482660378773, 0.22077327680195624, 0.34999097790306055, 0.88541548722859509, 0.77394736541191489, 0.82491585220013641, 0.91829342633121691, 0.99338103841066627, 0.034424881558236975, 0.088717627738795213, 0.70386703630691827, 0.14784074773715616, 0.2549122885099987, 0.36821168333158616, 0.81459092820738643, 0.31124986697308243, 0.024660992660288317, 0.43529806747249]] [{"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}] +16 [[0.88314255956027632, 0.79571874738813719, 0.97614372903888258, 0.37526204580470068, 0.494410856072649, 0.18558609496466283, 0.53527235841937837, 0.5038418765180156, 0.11777341444796707, 0.52338620845911255, 0.43109683101318719, 0.580587279361151, 0.75325940279621928, 0.135287423037333, 0.085888492946933792, 0.073282267922493816, 0.68090685035952847, 0.41301815871956771, 0.027978199771674928, 0.90885506087912049, 0.90747424628037965, 0.737383184415786, 0.0005222574143753933, 0.77224015940942659, 0.57151532354385737, 0.44059565505221232, 0.51357166143958088, 0.80802396061704085, 0.76619889800477536, 0.058065299999449094], [0.79736035082604262, 0.57170591099298806, 0.58788682814914739, 0.22672937891252976, 0.7914873649413845, 0.6369779410242965, 0.47336764687500288, 0.84970745641833911, 0.88767383982335168, 0.48124507100827008, 0.049774725006147658, 0.56600718160898111, 0.63303540777125555, 0.14368044361516696, 0.7886017689269621, 0.13920296277699651, 0.496316977686446, 0.67842809921741043, 0.7288606060018582, 0.11045510134680092, 0.10129527308124719, 0.61013771286010232, 0.41430928333090156, 0.94893201199322752, 0.039562565620639978, 0.43983494730554462, 0.97581133560278754, 0.97664372607972172, 0.48916215352245929, 0.74620236780263727], [0.98226965143667122, 0.54488976070263673, 0.38605475481159079, 0.58066546463404756, 0.28221163832961094, 0.29597057294519025, 0.8221518817496194, 0.66424863816689206, 0.56298456600724489, 0.85650595997180523, 0.67666108552809734, 0.35998719689027392, 0.76565341099525874, 0.9776501571617342, 0.22674505873656825, 0.51675028311454718, 0.81107601131518858, 0.96268810958463935, 0.62900441588730016, 0.058651448643713411, 0.31626295714491659, 0.20394274153900971, 0.71505233078109265, 0.62879273607611075, 0.11001310632758698, 0.78861874148487865, 0.50245643970363474, 0.68690918489834973, 0.39294759818145153, 0.42715305949957683], [0.18658144359949791, 0.20755056087058976, 0.32944871048842883, 0.40783841025851342, 0.14937384091708428, 0.5287577693110117, 0.7076425829432631, 0.88793045724835085, 0.69788768911323484, 0.96076136765761311, 0.058447103158285163, 0.59414570404425715, 0.82835838914130988, 0.48005007691260071, 0.285839981186071, 0.92440142761381, 0.30088905133080712, 0.20622774332269556, 0.42788409619989332, 0.51784519065415424, 0.70532216538823944, 0.387396897045597, 0.64168031137228632, 0.59166594394400451, 0.00062679520934627586, 0.30437811387169045, 0.885300779010772, 0.73087491776626845, 0.21813925976259341, 0.84489411220066457], [0.05588855096657297, 0.36118888695641416, 0.63325354250655719, 0.49591804128105843, 0.53549203942464207, 0.83534452353256772, 0.7306650780288646, 0.15570590374188487, 0.19483567452202011, 0.91961756314276555, 0.715029317737101, 0.47660302884219086, 0.91011124604729265, 0.48001521138549874, 0.15190472550410306, 0.41553503061918196, 0.55900671955501835, 0.336556987607466, 0.56310727296799234, 0.29116413628073912, 0.27957234720737789, 0.57925281674972173, 0.44611137421102087, 0.92501163322314728, 0.49915712899028986, 0.12083375857741363, 0.23889025256991081, 0.37841788847112279, 0.87053049641630131, 0.89898546531165957], [0.20959617860231794, 0.38803179474534166, 0.78843812430500582, 0.93477996549384124, 0.11473927335965362, 0.84959117553013808, 0.60222252657289732, 0.40419661096907389, 0.74003083205525944, 0.60307693221967051, 0.98877261228428026, 0.98606842731052913, 0.11481549103256239, 0.33189300808944933, 0.20494870318203384, 0.01248519898248468, 0.27402405377592076, 0.60454554018055429, 0.099762863804714219, 0.36663042723756512, 0.93083654205563526, 0.95936402669656362, 0.83397660943626739, 0.64822383930819261, 0.31875323987316473, 0.46112830695942686, 0.22841120678795712, 0.85818724800556689, 0.76352874996377129, 0.34303502564704158], [0.54214880949651245, 0.74184637055627034, 0.25343254589002484, 0.18825786578496106, 0.267934800377606, 0.9739266068343182, 0.39055650991295243, 0.60242587448528107, 0.22447532908537016, 0.56879622854450662, 0.62639870160778222, 0.48166472969977736, 0.8239561571645031, 0.25585556985221714, 0.61170967733113712, 0.790416060223728, 0.8979922495517918, 0.072666426196961731, 0.82359562424473776, 0.229070173270136, 0.87635011543530483, 0.097360694973200324, 0.37213083331421382, 0.29351794294520517, 0.4301676109829129, 0.067062998342373947, 0.0342393022283175, 0.0070477665635586551, 0.91502905492925046, 0.61050730335250647], [0.11436318555256575, 0.039584178806211123, 0.07913603228044408, 0.96072259150120543, 0.68944412742122341, 0.020084495734236096, 0.53794719963903626, 0.30760839745994406, 0.16000044844437145, 0.59498051486459136, 0.45904313201874392, 0.40041582124856856, 0.79959181270414115, 0.188872870217362, 0.74960539135879034, 0.616595222749575, 0.17172021368983215, 0.48325501486866029, 0.793781046243252, 0.47616745722143183, 0.86622431866487415, 0.34486534575302141, 0.44833775120975983, 0.499719923909254, 0.9964817303393575, 0.30733527060846544, 0.572396328192084, 0.38332488699399181, 0.29424187414235503, 0.73812197895303577]] [{"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}] +17 [[0.88177831777895144, 0.38759617346917141, 0.11526365564396113, 0.72043868520836585, 0.61138479638383181, 0.9620115658850783, 0.0084347806470101983, 0.484256749062121, 0.095957799189622328, 0.50705669354063065, 0.28536282230644472, 0.031494876720769782, 0.29391281021556992, 0.88795459111977881, 0.091578864840146212, 0.28793303966902617, 0.96837305368696081, 0.920353399564873, 0.90675818334837543, 0.046597266979181429, 0.26320971953020911, 0.95490229999020537, 0.37673278661768461, 0.71874262887924723, 0.0099980594643507281, 0.35302903108805439, 0.19876192272915494, 0.017252063855499378, 0.92351298803199611, 0.91999049193889715], [0.13809390704355751, 0.53158813929148119, 0.78576999793704538, 0.81146116698775883, 0.02436080949620989, 0.55675514012742022, 0.56725041390237241, 0.036371636799054308, 0.753104616999493, 0.59829877518963637, 0.1888326993063203, 0.44140279319319609, 0.90074210462674176, 0.69119169229918165, 0.28908445506143088, 0.98148816009188977, 0.550533064323865, 0.042522859563579707, 0.92895229961475789, 0.1558710309955974, 0.32262425718187127, 0.53622932561168024, 0.6395197674367602, 0.7207092620205251, 0.40299954931027404, 0.32267933480404032, 0.59838662151817623, 0.17691506366546927, 0.57417432761310683, 0.056735797605043659], [0.0063546358628056021, 0.075863069698958951, 0.8647300514312749, 0.34499966991365083, 0.92848931788784672, 0.88579503297555617, 0.45529291749578282, 0.61625289474060441, 0.96225815815090165, 0.01819480829283493, 0.90531795154660588, 0.93869126317352836, 0.10033025321560973, 0.626102548465399, 0.59804811884630271, 0.27335727956458322, 0.0667540482176242, 0.95528257454369825, 0.54027544971423647, 0.27525386506376315, 0.22261307997914581, 0.35435443948531042, 0.2990395230681554, 0.031940043690750208, 0.860892464326198, 0.53144195714701348, 0.90915563110488129, 0.0639797751010891, 0.34017158179756124, 0.28259095164919745], [0.043473795232956691, 0.50243572212798515, 0.042891138110540905, 0.711150528662088, 0.602026687588878, 0.60566970247055274, 0.58632585229325251, 0.90750954834747455, 0.14673287609322705, 0.6810324371747799, 0.063026755951264612, 0.16502372909565521, 0.28507946356291181, 0.036594902553426012, 0.342379456807115, 0.0706655264000805, 0.18921598225703207, 0.80389424778156926, 0.62605842716675786, 0.22787315838668198, 0.92614931949542545, 0.3447374866787587, 0.51753914507196153, 0.94427097157928785, 0.17023698101125606, 0.48342676910590665, 0.83946384041138467, 0.94837882028589626, 0.17093423426683718, 0.99033946504674952], [0.31253807572669878, 0.087122868201550241, 0.55675812395912094, 0.58361781616420261, 0.9957698309852121, 0.99990614859308313, 0.6706894766817314, 0.21279232329378817, 0.0802438749069706, 0.96519204498026656, 0.652741350880272, 0.54148365201730608, 0.22725838513039653, 0.47199949980667832, 0.41218089388901558, 0.3558981769350833, 0.1465328697744247, 0.58140392780090067, 0.96489864716794194, 0.091652691251193907, 0.52132931973090446, 0.812083069808577, 0.35067767607843059, 0.79353367085537774, 0.70471169256044064, 0.10914247613653161, 0.49062253959164581, 0.60111758936733384, 0.83480108817378806, 0.43690056846139813], [0.89676583307612245, 0.2877480778656013, 0.14676743952428584, 0.67400841532630218, 0.10387799767308004, 0.47118705475392963, 0.29543069001837319, 0.78322495991841046, 0.057763845704690953, 0.86132654742912185, 0.44472773686372868, 0.6783632978970372, 0.46483704149051419, 0.31366850011712522, 0.72893412778099931, 0.81736952056697509, 0.28452154126590878, 0.51685015974112392, 0.44541193523097733, 0.31319026028133579, 0.12903718163832067, 0.85831308611021484, 0.99978866415314871, 0.2730312020307436, 0.32438354425281724, 0.38428368411071023, 0.19185452412165371, 0.15180306026647494, 0.78184526782841035, 0.7985541175777443], [0.90409794298099722, 0.13699809504385096, 0.63109853791390735, 0.12912233175080534, 0.63468251678089427, 0.96993446269310568, 0.77400741737480372, 0.032570129085711219, 0.60280136888525326, 0.15225160164273088, 0.13245550151157781, 0.70312910973061471, 0.346727144385645, 0.18673523836311123, 0.3840669651265548, 0.0067039703320389554, 0.83350915902660594, 0.35799508913705269, 0.40642841966789767, 0.76969344779752913, 0.10415224344659424, 0.57121513424825365, 0.48343108373851529, 0.1182691014486621, 0.38924381369513672, 0.93613987917456554, 0.56661032316606341, 0.38584290598890225, 0.44737956343010121, 0.10865690859562338], [0.52986772527251447, 0.38477437153649419, 0.60236990351675479, 0.8837802336026408, 0.40771834386509187, 0.77293880610394639, 0.58509301346511444, 0.17153484117558337, 0.065213942800372848, 0.83659833398242245, 0.82199279329514052, 0.281188123386073, 0.61195060584323491, 0.25808727458120806, 0.2546024700162115, 0.38946876011415765, 0.14307684812055432, 0.3401447965826967, 0.12310816302935357, 0.17485947953683256, 0.066976583273594792, 0.71829680062218948, 0.40184893341975225, 0.86274513711658674, 0.36230856973368064, 0.8085103545480512, 0.73578295357617074, 0.34514280917288032, 0.59634203772006844, 0.10941371515019105], [0.79433200534156856, 0.61393616586413025, 0.4320363405288411, 0.0066954926562164951, 0.93264960494789129, 0.47798155528676822, 0.6530695022554257, 0.91186673169784271, 0.93024081195150721, 0.42214835303909715, 0.44402904207301808, 0.40839031010096605, 0.36299425196757718, 0.51030177624513484, 0.77723610309248636, 0.57525169404208176, 0.85707006757956739, 0.43443166342005535, 0.27818135730730575, 0.81136629784116343, 0.071720544106790407, 0.16857562207956611, 0.86830954456916754, 0.29378808146197333, 0.76300196406469645, 0.19888065164465019, 0.81689692559746441, 0.66850524435069314, 0.691470372625984, 0.97009274238347554]] [{"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}] +18 [[0.8092757084352763, 0.08992259457487628, 0.18569280785172559, 0.75611467712240266, 0.36377485165404111, 0.64078001567212739, 0.097821453818684456, 0.75786548388710884, 0.46809393371445174, 0.77916698053229594, 0.71831147668606887, 0.2558786610182906, 0.65143035479830069, 0.88448044626726907, 0.15510983657522392, 0.10539377936596794, 0.024019638386299502, 0.0038249526301865622, 0.0799957636382671, 0.69475784262231544, 0.01944506732524065, 0.04491472639605254, 0.24990754910401825, 0.21855671163441104, 0.72700207742480361, 0.46425005383399554, 0.93492835330863366, 0.19731787223759789, 0.33721860803914361, 0.51688318236212616], [0.98159929440276161, 0.14457216156888775, 0.29882922836494563, 0.529595895297734, 0.2685485367831848, 0.75569447986567628, 0.016945004337063585, 0.46106895291888095, 0.99599504002867068, 0.37282672820396279, 0.30985679714106173, 0.22063209358255176, 0.59150599600025722, 0.76631836483711191, 0.3858556195820757, 0.83143987111478623, 0.011457893004749531, 0.46550349265024449, 0.80164771978771665, 0.5444276053375412, 0.87097039614495064, 0.97026581905499221, 0.72038843000784025, 0.4361587455828706, 0.14465348787672416, 0.17972564262956858, 0.80005389062375154, 0.85668307690773371, 0.14632832390827566, 0.28551597954130048], [0.21181460877739855, 0.33278309435782916, 0.11795215939927239, 0.19449767474391666, 0.61327077838184618, 0.17749335322238369, 0.036298740644152394, 0.26939611634089167, 0.38285409380014923, 0.10853577545972493, 0.18051532406707838, 0.8057251055582938, 0.059396504442058573, 0.63632147285585949, 0.59449794436875991, 0.19949608244103834, 0.724811104380503, 0.84642755987718432, 0.82509706629785651, 0.20259395412863002, 0.018026588524067244, 0.4719652265590133, 0.22048564921963343, 0.011333279607224855, 0.57476084371790925, 0.82801308373847093, 0.63259242053810016, 0.18859159015660154, 0.95041561562161259, 0.26329774771437919], [0.58332319867075266, 0.58104949776115178, 0.50231886642181378, 0.27356130725972772, 0.82636338235160112, 0.46044969631237642, 0.28273696247863755, 0.31656320456249043, 0.76467527525185974, 0.0914204677115662, 0.7255241319944038, 0.67042185020354583, 0.36837404290394826, 0.80829099106124469, 0.796971978951656, 0.38500901049860237, 0.078486309332762239, 0.46945996915162058, 0.325367037180184, 0.4903682345796, 0.67916612577221225, 0.850260691503135, 0.42395978490958874, 0.14663555523188554, 0.27426282306099214, 0.13389914746885923, 0.27849381321369004, 0.34669960548618783, 0.31370597031573455, 0.16492429550575405], [0.083139040762695848, 0.11584502956856813, 0.080992949266478931, 0.540822769522827, 0.57761840950102961, 0.8918818341645407, 0.175134456604066, 0.934712476375835, 0.46351269580965027, 0.9147703093581897, 0.29873451788379735, 0.26574330290439818, 0.95917310820552337, 0.0065095816153252217, 0.09845092358855434, 0.46369476620677808, 0.89036733326470141, 0.080065329803413743, 0.89490279324345989, 0.61039445197427111, 0.00772917753765523, 0.080370234299783627, 0.88616253826072222, 0.18296303721400931, 0.67227798949410744, 0.83093813173939846, 0.81228906851829641, 0.972524923436018, 0.65671884547250425, 0.43639134242899524], [0.12793016003335389, 0.00690612192021145, 0.57293001416971756, 0.47420775031285278, 0.4458416402606471, 0.46697409199404027, 0.35498970052919143, 0.1982524500119669, 0.38624198672408017, 0.1536567081616238, 0.30228888612178573, 0.3721664182394181, 0.99524013834841341, 0.53631065922678167, 0.418722942896547, 0.15244273109919637, 0.58014642125814975, 0.79299209564672524, 0.17164966335760812, 0.64902806780499167, 0.15029456291233767, 0.53931550683897311, 0.33824664469359378, 0.40554585038363844, 0.64907522940334017, 0.13496939124495211, 0.3070968255658727, 0.061744581437887636, 0.81620029413520157, 0.55558996408313288], [0.059275211681054762, 0.887532903802175, 0.82878812217989173, 0.82257730343147117, 0.065906314233182051, 0.71097094638080482, 0.835680314736603, 0.8945709661855763, 0.50810642127931716, 0.0018450137617092022, 0.12147042185428414, 0.74168489699882112, 0.55042031262580149, 0.51796060682730716, 0.91504594828690666, 0.047234137302644807, 0.88030725156080369, 0.97304675500795113, 0.7920146545870983, 0.61156920769112411, 0.36392681663225379, 0.64317216160120372, 0.21120713709718864, 0.06876599917092574, 0.46241968980766712, 0.69283160578140646, 0.97251742316997469, 0.96930263666522354, 0.0030746653796815959, 0.27236891681037168], [0.28666386830257584, 0.63048691267187873, 0.0062178023579853026, 0.11516063753153527, 0.7070230958259095, 0.41128142093742182, 0.53135538971560436, 0.44599209137612095, 0.38950967502825273, 0.22930548408806406, 0.9250473511889209, 0.5013157942957881, 0.018545031208548668, 0.20517656910675219, 0.42818338938359746, 0.857589460541127, 0.26547864989218195, 0.29298615503009517, 0.83177915700363425, 0.065649086773193366, 0.83039609164155626, 0.63558826794081524, 0.75810534249910366, 0.79708027250993174, 0.95906717340529457, 0.16951101328531681, 0.18790754642448082, 0.8228086126149492, 0.82385948510868612, 0.69668533537962851]] [{"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}] +19 [[0.19214938505481793, 0.1203246770444657, 0.22682061095121742, 0.60058123359852333, 0.41666223300525729, 0.10859332941909827, 0.46470160831520668, 0.86941153845249852, 0.45663333493050606, 0.51086711252431372, 0.50942562355559862, 0.89459441848345933, 0.41325362721123637, 0.381879330677928, 0.052761983830012515, 0.73072261693733631, 0.4855053961100595, 0.59134011258518315, 0.82286092628847052, 0.30720833022562843, 0.8581933488157647, 0.36446421974453491, 0.52815704589312573, 0.86886456411555091, 0.28842875010701974, 0.19960917399368416, 0.8007055942769391, 0.90426876784123877, 0.63241830605010407, 0.69808868356172171], [0.086369220452555684, 0.43144638484416, 0.62960485862541093, 0.23673376625685194, 0.45759446613749444, 0.98404444540253944, 0.49265244243139961, 0.21553183890854222, 0.0090858694843565235, 0.021047988515553895, 0.2394633698052645, 0.8206818125551516, 0.83750786483854645, 0.76922544470779941, 0.24736045418137353, 0.85766164818117929, 0.60341307398947241, 0.69987124546558854, 0.20976271683972492, 0.22941129526019721, 0.3269172336928805, 0.94004724789595806, 0.5321280983888782, 0.061493491542775169, 0.37821090313289618, 0.92340721226670663, 0.2166326329449112, 0.30910109524046736, 0.58981792729369176, 0.4968213500040467], [0.45399972171368297, 0.92003849986866271, 0.039581768829315411, 0.516400676778439, 0.4660897805866786, 0.27331206029650257, 0.33168060954940548, 0.69015403633605321, 0.62430470113660175, 0.66472242217370636, 0.13754245552574784, 0.05439322225641563, 0.5065577263960368, 0.068859242187711445, 0.34935211865133053, 0.0071342724872612839, 0.34950395554455826, 0.27115683311387806, 0.44591177466262821, 0.26412640097649254, 0.30292269064536481, 0.73417706500207258, 0.57393757150431979, 0.4517721503317077, 0.99958757145518307, 0.23904340209471409, 0.4074448135434573, 0.306085747288546, 0.21665605684802203, 0.42595823767098873], [0.063694515300421362, 0.60796902408130338, 0.82355511145147142, 0.84699284331426061, 0.22624481167306354, 0.052467347216412441, 0.78653510760639744, 0.039847927511421632, 0.15397045918143726, 0.22330071266903595, 0.7315095368723904, 0.43303963335160489, 0.43948434209974352, 0.72384794038768341, 0.45449309893086565, 0.60097443228065672, 0.2212153014755317, 0.30189480570725125, 0.81289632078054119, 0.50163376545131155, 0.050466584952173044, 0.49783980202325651, 0.63113007209946781, 0.85009727902995924, 0.76809963527651648, 0.94468363553794843, 0.610418660401025, 0.70789085808285912, 0.55505848246506351, 0.95301202897086468], [0.74541544822556072, 0.70115454603601735, 0.78677014237715226, 0.42531379616389164, 0.41916032779971568, 0.01988171556204299, 0.44352643602458475, 0.4613587995096492, 0.17311378225150942, 0.495990209436561, 0.60727369162444345, 0.239993823731537, 0.13044722445727974, 0.60974279987447078, 0.0098062674723040644, 0.89259652148264257, 0.18846508661411565, 0.23204287126479872, 0.68818088232168673, 0.11956098640980573, 0.38997903694611491, 0.2743886444328909, 0.55023842557362879, 0.24357077376035441, 0.33365046800050846, 0.18877727400608879, 0.056440745602801434, 0.24204862697234497, 0.089762007212484174, 0.43660772106254508], [0.28379985441569733, 0.056054337194134884, 0.8505750015302489, 0.628639667798844, 0.35349754556937474, 0.13403529649995305, 0.46337443073785456, 0.027322649385253639, 0.57586357169592972, 0.73621755533863709, 0.069498563430633431, 0.49831839815018353, 0.69470102803750267, 0.99905668292655414, 0.30409572471836244, 0.79979638655629492, 0.29751750032605639, 0.13905243286076296, 0.89904600264856593, 0.30609086224471704, 0.79904149826911575, 0.46136201242965513, 0.075430672085835426, 0.87656538338233569, 0.40247342994947921, 0.031031436242929034, 0.81212816967810464, 0.60103826877220079, 0.0962407253575791, 0.96329575602531581], [0.933559810169161, 0.57988111380647345, 0.053597703950848508, 0.29428259096255405, 0.97230465405583255, 0.7299091034832903, 0.76823004719167831, 0.30458620760985455, 0.93124745806072518, 0.84644095633806971, 0.88109476932603181, 0.985561713095734, 0.39511975625373941, 0.79741296636712833, 0.93204706854855168, 0.061822011677483135, 0.92273841854193506, 0.77719022929420534, 0.20437464109897452, 0.61330345148339249, 0.28894274658743713, 0.0057257944187022458, 0.42118194626006922, 0.87639779717415833, 0.93917470707151751, 0.12297585924153642, 0.2996357539297535, 0.24603214977122656, 0.96984398902346369, 0.7989876665031308]] [{"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}] +20 [[0.22723015667366742, 0.40942309462872317, 0.28492576086201182, 0.79017305587450148, 0.24232704437415198, 0.11447196385870251, 0.38825996621023184, 0.43351441214718966, 0.14355702268519221, 0.24975457610358065, 0.91170638842166751, 0.49546179118897504, 0.72433415000646428, 0.20938732646353675, 0.57192029994015869, 0.33054049354602955, 0.54942672554174132, 0.70097306844279772, 0.228089286079883, 0.76484567032571693, 0.33713079989145645, 0.67534215083811167, 0.57934412010664738, 0.13377981608827938, 0.46725714542337915, 0.54922883667486577, 0.53493463209850645, 0.23792403525811034, 0.31552358176190332, 0.79867240656094174], [0.56567235513719571, 0.37539872409241604, 0.43232796169442056, 0.82612756797862807, 0.49065026550726776, 0.38738979167825638, 0.80750897977085057, 0.98571408184846765, 0.7050072490009438, 0.088086570842380474, 0.59258559282853818, 0.788730912286281, 0.81251315400960666, 0.5410983949494893, 0.68809816897022025, 0.60550527270206655, 0.99100754138008662, 0.39148633826982548, 0.546057774134103, 0.6298426647337062, 0.1892337611007664, 0.8658678757532059, 0.38614201324693964, 0.47878910446951783, 0.43036713925353687, 0.11262172578203877, 0.77915959097907439, 0.56311166492769726, 0.080544259310863175, 0.2700408770227658], [0.2124380415750714, 0.28284968024021884, 0.929958183964707, 0.022352113824621989, 0.81917165645179668, 0.17748316227718919, 0.32959183823415505, 0.4318403033312832, 0.92740033180152437, 0.3917114213759163, 0.42842007722166642, 0.51988661777819667, 0.46417182647392563, 0.20552919336683129, 0.76391855344611925, 0.41054960648012928, 0.53657350312912067, 0.92123355501199411, 0.072975347749800967, 0.84738851111834446, 0.70370201301976876, 0.10018996648917156, 0.21080886811201527, 0.54754903278348432, 0.55227242451743075, 0.998630955746688, 0.49225404870510148, 0.16670107697586212, 0.19345098926936632, 0.064456836566695652], [0.60278589380428671, 0.61423917617072454, 0.209628534691346, 0.52595229914114483, 0.46590753031530607, 0.080226487401504287, 0.39723448657927396, 0.42852873892011867, 0.099038145434151681, 0.41388281242469438, 0.37532406060283907, 0.9570472632389202, 0.5417931213948276, 0.42798867261474527, 0.75955405079307969, 0.51897261296010988, 0.39855746663744873, 0.29439654494216438, 0.13729570803871249, 0.85755344515518817, 0.40151073556404826, 0.44291150386925238, 0.52201562249949085, 0.41441991522684452, 0.67076970955208759, 0.27912345340136213, 0.31368864183320821, 0.267434810900643, 0.025073017775296913, 0.59634357258956716]] [{"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}] +21 [[0.80210647334191454, 0.69884789048694707, 0.67325247016421774, 0.18647318626057185, 0.71527136491642729, 0.96996035465464525, 0.86410361218641141, 0.43629253812037783, 0.1343664366155537, 0.08933635028895226, 0.26242879032691524, 0.96754901252159808, 0.31618543964171286, 0.783798165587592, 0.95305384434963836, 0.547585035257943, 0.0026095191527407691, 0.81746370905381027, 0.7398086347834486, 0.33751581121472762, 0.38505421156561148, 0.68032046740996477, 0.1327005237291784, 0.14702136790917075, 0.062344554245973294, 0.76303329680015064, 0.14792829349608072, 0.9089687593524568, 0.84690744774636639, 0.29694686594724218], [0.35228559989381047, 0.45214271127601913, 0.02905277297607789, 0.10093975614127615, 0.16651087021904898, 0.36618207228249966, 0.37024804851804161, 0.095731636375122386, 0.43139832867640382, 0.12489050877052621, 0.6636737690090182, 0.6214406422490939, 0.6359925343588686, 0.66240999411188428, 0.17397970770203552, 0.82563366392706627, 0.0089470450303853477, 0.28131582270999966, 0.64804515823834785, 0.77680712519867923, 0.882135053883447, 0.11251236565843969, 0.87467883890946474, 0.50218111527120635, 0.34985802333919269, 0.34387904030560157, 0.96357713629972552, 0.84888570040277445, 0.26750619321735958, 0.087936169930814767], [0.61249201752696247, 0.9240625842157949, 0.060282444734582885, 0.068371083488376949, 0.63644300659874808, 0.88210154997210866, 0.64560883362902888, 0.27108438435765536, 0.94394007249444145, 0.99617563956378663, 0.012093180158374128, 0.32748764737109237, 0.962567291660455, 0.27643503602774766, 0.64344046710238345, 0.67610314488631107, 0.23991024883071888, 0.37866421330192512, 0.027452162854329387, 0.483087235910321, 0.57569220986911462, 0.18490285542045171, 0.94826388394583694, 0.89311876723853445, 0.57711762716961656, 0.2314760028140882, 0.12422993236488866, 0.42916478033746297, 0.97180418256071077, 0.11007440556067438], [0.024370861593353554, 0.26295556338831882, 0.36131688598630751, 0.073971034731617591, 0.37305838448978113, 0.63757170578068412, 0.35320141430705987, 0.34415547941004865, 0.93561932467315412, 0.93936605070340184, 0.17054594295381098, 0.029112944089999981, 0.66679819752251324, 0.788513851666853, 0.9054838313743675, 0.147169861689734, 0.19952622338308434, 0.045804454562677344, 0.55668258346114818, 0.97177382583197947, 0.95320193196886283, 0.81785852110738977, 0.87074663726199331, 0.80777031105712072, 0.8486963543790379, 0.38377496033001235, 0.79359524765461142, 0.35068427938169544, 0.7315542219093093, 0.4632817891856269], [0.030258673446681428, 0.98121029751838873, 0.12536918495094362, 0.39978604017980524, 0.81016934480398017, 0.70355952568166724, 0.8640703755844974, 0.6412874571083037, 0.459907030502545, 0.9490076445522917, 0.48486434427696312, 0.36794178166355906, 0.79649030360803885, 0.13493323976896088, 0.0028343488802465844, 0.23712880449053464, 0.67168885185289862, 0.69483108013421124, 0.83370758176008619, 0.995201848986611, 0.22146461548252239, 0.27623009004106913, 0.18305758065455835, 0.13966442987932892, 0.50453190198409914, 0.941141583743334, 0.17821786368835335, 0.90634110074184016, 0.25016011669774685, 0.49760814300010103], [0.38343997395809915, 0.60317941374138551, 0.37772490843271467, 0.54111895984729064, 0.10933737096998908, 0.71564008528756129, 0.79914678876079148, 0.39110284733212952, 0.10441410230867731, 0.26820313127314388, 0.770239033309342, 0.87040559200233159, 0.799485270350349, 0.64233783239719777, 0.71811919624962128, 0.79460321221790953, 0.26505388781829042, 0.42335474809314044, 0.0392217444407168, 0.11319826254879861, 0.43721671638816384, 0.87687814001620057, 0.83608199795620841, 0.87619194588588156, 0.96942925501635357, 0.34024797204380897, 0.94094422582659754, 0.58721926933364754, 0.27546832167643, 0.15498035007071309]] [{"5j9i4l-WIO63j1-HQK-05yG":46266, "9T4jcW-qmJo8OK-vDm-SEKJ":43268, "H1lgcj-Udlhete-kuU-TF4v":22715, "IS4MMo-1Vby7UV-WUL-lyGu":3583, "XBgqLf-4Nb3Jst-mC9-FiH4":61532, "bXNFss-IaMgfzI-Ih5-gN5Z":93603, "rnsdUO-GTe6ufj-W7b-WqlL":49324, "tEedw6-XWQbwOa-jPD-bfiC":82421, "ulYVtz-Hi6kCeV-DwM-3evt":13356, "wZRkAe-Xd70AJr-fR6-nR1d":94383}, {"5j9i4l-WIO63j1-HQK-05yG":46266, "9T4jcW-qmJo8OK-vDm-SEKJ":43268, "H1lgcj-Udlhete-kuU-TF4v":22715, "IS4MMo-1Vby7UV-WUL-lyGu":3583, "XBgqLf-4Nb3Jst-mC9-FiH4":61532, "bXNFss-IaMgfzI-Ih5-gN5Z":93603, "rnsdUO-GTe6ufj-W7b-WqlL":49324, "tEedw6-XWQbwOa-jPD-bfiC":82421, "ulYVtz-Hi6kCeV-DwM-3evt":13356, "wZRkAe-Xd70AJr-fR6-nR1d":94383}, {"5j9i4l-WIO63j1-HQK-05yG":46266, "9T4jcW-qmJo8OK-vDm-SEKJ":43268, "H1lgcj-Udlhete-kuU-TF4v":22715, "IS4MMo-1Vby7UV-WUL-lyGu":3583, "XBgqLf-4Nb3Jst-mC9-FiH4":61532, "bXNFss-IaMgfzI-Ih5-gN5Z":93603, "rnsdUO-GTe6ufj-W7b-WqlL":49324, "tEedw6-XWQbwOa-jPD-bfiC":82421, "ulYVtz-Hi6kCeV-DwM-3evt":13356, "wZRkAe-Xd70AJr-fR6-nR1d":94383}] +22 [[0.33418430452769021, 0.1571443475012273, 0.87139187590553191, 0.763090618220186, 0.47775193122212678, 0.28389999727885518, 0.60928065723065294, 0.18254287191968643, 0.57947007579527943, 0.72593937626820182, 0.14287710926962827, 0.87271028812930207, 0.77124944454399147, 0.24053102139362426, 0.29016219903811691, 0.80315001498314653, 0.79098402862753814, 0.80971295720380854, 0.20572852915186635, 0.26142880150871572, 0.91710606956334417, 0.66677239944332367, 0.88713410049022545, 0.72340865706655411, 0.21737001845386472, 0.74304149549426579, 0.82270978751443569, 0.69775761928133251, 0.10737951478765151, 0.57959476319269088], [0.60299469128615713, 0.63858798321101629, 0.46925383993293845, 0.011130142791301534, 0.40098111747672516, 0.2359452832133756, 0.741963769539048, 0.34826007223526012, 0.71560910914008236, 0.26594076058061633, 0.46224879928794427, 0.35963507130215977, 0.66845763741676445, 0.7106414366916356, 0.70866122571945223, 0.086767069803651009, 0.44533823593898458, 0.042400966544069996, 0.78116483107884471, 0.41211516551293337, 0.18250392727391163, 0.40813589474299505, 0.23484915367653525, 0.34494203782165722, 0.71144439958151928, 0.9461217520263413, 0.0787026455601817, 0.18964430709762137, 0.096746371583170454, 0.62120335863466469], [0.54902166255379914, 0.11770550707253591, 0.82825553160603194, 0.77388503768587935, 0.37191383025405189, 0.17711562600570296, 0.86178445457242581, 0.044585536409431636, 0.8404884133437629, 0.4420466996702872, 0.44186889329586521, 0.45641383558216553, 0.34970829502918888, 0.83076879642307688, 0.785582121873371, 0.21814165957513088, 0.25551116287344755, 0.10123665028096873, 0.25817551647769543, 0.24361883975839349, 0.054690513307020017, 0.67147184745670674, 0.62932066984889656, 0.47469873261954509, 0.9181597571184491, 0.2058113884000603, 0.36298676227366822, 0.84181002822668294, 0.5526054643978906, 0.35828844057795095], [0.6644942845382964, 0.09963267693329747, 0.21416857011840207, 0.67085632929860239, 0.1850441663748178, 0.83490522956258062, 0.45267569233542204, 0.38686234444523804, 0.86702943045273806, 0.65921175796600728, 0.82607449280006051, 0.85912326339470313, 0.40269374312854067, 0.97166625503522186, 0.39211087049993376, 0.3071496344579735, 0.17420957630080303, 0.70473339014304748, 0.25791814598363616, 0.90951108421255578, 0.345006267184896, 0.15099416950029243, 0.84807081801231909, 0.60503072764194732, 0.98018613750157912, 0.30948576052223875, 0.83778928031758748, 0.52044145842104861, 0.68812448892766553, 0.18731499488616021], [0.89036346894801377, 0.64233269826174355, 0.29585735353310816, 0.14556525395910935, 0.43649747862395472, 0.20766095809371798, 0.49761861617043157, 0.688086845805612, 0.738158837209994, 0.29979369418777635, 0.2488957937091566, 0.057581232518764125, 0.39064986100691368, 0.3060004041967106, 0.9176315026313453, 0.6254600071904004, 0.23400063457263676, 0.21750051781164526, 0.84970624668501438, 0.34027924876054061, 0.36841664743632352, 0.23462341083308402, 0.057594406280263022, 0.43412109504206897, 0.59839613024996152, 0.777330379424147, 0.19909502256141864, 0.17711379114807946, 0.16626588327816816, 0.7952944545670152], [0.8772950486445692, 0.717352400414326, 0.078266125566924583, 0.22192857975363289, 0.56343186409376067, 0.94680227451033627, 0.6749488157830178, 0.5231986595219259, 0.12107880091446066, 0.89333805690933732, 0.35253590775876587, 0.55475285390756834, 0.993085741744833, 0.35489100872771373, 0.50301693643176848, 0.35984377903828368, 0.6019644373613654, 0.36374784123056325, 0.80814873610211513, 0.11571603655743357, 0.4099159749574357, 0.5973647170078108, 0.99615193128062385, 0.16757613990199494, 0.38645584070381678, 0.036630120414640532, 0.049782687149953819, 0.97695796687433811, 0.33959966381245421, 0.013330761608059905], [0.64059000868999438, 0.85524148441456582, 0.68083054114091157, 0.28564515016570746, 0.2565526283409767, 0.553278196197941, 0.71457679149234132, 0.93380928329130075, 0.088921630512443173, 0.13452218574330255, 0.46968665995218728, 0.67766994734202668, 0.24897810950254651, 0.071760439297789858, 0.42339434679406673, 0.68366061353723806, 0.29654798946447913, 0.17648194318608135, 0.20752951936365815, 0.050806008073532416, 0.37949871054978712, 0.79512025025328481, 0.53223052163765971, 0.94368108383944493, 0.50707551952765129, 0.68784915711747863, 0.11417878177368745, 0.77482101260376846, 0.60981797017613693, 0.14330709477937864]] [{"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}] +23 [[0.21512619243621667, 0.81893079651557932, 0.98440744837687078, 0.53843802238875649, 0.18543355132489914, 0.0815671479746668, 0.35961220433253716, 0.61281807480648465, 0.66739946024083285, 0.78675272573864563, 0.24811238439506289, 0.30348528982686418, 0.30290214563319373, 0.6545970678746591, 0.37749638044505829, 0.72370899826241708, 0.52684526242281526, 0.6285903609383352, 0.89221266061764115, 0.19832330352680139, 0.9205528516890793, 0.11050075973282347, 0.29233630106450381, 0.5136567466714872, 0.21159225603577536, 0.92617995646459228, 0.24320486336590186, 0.035043182561499076, 0.37794153585512291, 0.36910969622694934], [0.56393807650058436, 0.54900032683837252, 0.99117535875164009, 0.22809122741083965, 0.957737645312871, 0.94985727097492045, 0.016351020940711836, 0.92130119550032952, 0.20799697651305005, 0.30955781610426292, 0.77553465053541126, 0.73100667700197619, 0.63163299129984107, 0.44456195564327716, 0.67741490543653526, 0.66155324322091236, 0.55468534334699, 0.8252124820689718, 0.617209535294706, 0.6278097586683552, 0.45511097084795749, 0.560630037742666, 0.65907786542921354, 0.52207397831518088, 0.20383465211537288, 0.23614406930703902, 0.72751032572407281, 0.20451353916514037, 0.52749941880984985, 0.58046363966043313], [0.55467356119742917, 0.61553716787209689, 0.41499301282485856, 0.2962362218766168, 0.057447943074404151, 0.018060877170638867, 0.69470970620953376, 0.90033902034305258, 0.65090157223942746, 0.84345498553657383, 0.42063954617665167, 0.089191520784699052, 0.74420329157041187, 0.73702735903991246, 0.022759871352225458, 0.96957455658930747, 0.81048443057917663, 0.17721900910588362, 0.40083666666810713, 0.4768852169661939, 0.97391728977913228, 0.47725846063268196, 0.27062639991216386, 0.44045319622633594, 0.015286004737699921, 0.068841250837361034, 0.10760154866466487, 0.049489056984012181, 0.0615830366960366, 0.71447199062731315], [0.97570579742021391, 0.923430305401204, 0.995672803405771, 0.69842578600352179, 0.37727818894743381, 0.59689630435941543, 0.7988213407208764, 0.10585700856841562, 0.015167384420374685, 0.99215871511446574, 0.62189896691674762, 0.015402919862829934, 0.019191184943811, 0.37097928668456981, 0.37032064970558376, 0.80270077224874337, 0.756478271452453, 0.33286177746966594, 0.98329652300631176, 0.780556555439777, 0.81886242607961846, 0.65821087690035285, 0.38972290081899785, 0.086074527693317915, 0.16038224960383107, 0.18258569287285997, 0.33105206983866631, 0.0658763968087045, 0.64407215273626939, 0.4321806949989786], [0.85573641036712622, 0.40760989751213961, 0.44961318870913713, 0.45086276057171559, 0.3103511722133121, 0.91926617926625487, 0.892045274777643, 0.10553783341525658, 0.079256957235261383, 0.16739664363650331, 0.20662480560062046, 0.16931624007356416, 0.15767159667905117, 0.15113889798520885, 0.76604478410979016, 0.65089097432243881, 0.88227854516911131, 0.81103331433612591, 0.013617655258359007, 0.16373854288719059, 0.78900676832689, 0.44993062141536622, 0.11965280306085946, 0.61019733701802092, 0.61937494540720028, 0.07786208052332444, 0.9981449017892966, 0.093173762502698487, 0.95480068674719587, 0.28356404728874307], [0.85702102108798117, 0.76217164399041637, 0.82253319940789371, 0.48241869754674449, 0.89757769973249713, 0.25053585227230268, 0.88737549765826529, 0.29389300751513181, 0.12737175450249361, 0.46724625327922642, 0.52012051845943019, 0.36112924765816745, 0.50571189908180458, 0.76929361556858067, 0.11663287573527259, 0.14115899786891006, 0.80203039079302918, 0.95961396609499172, 0.53076573756457324, 0.0662846515089599, 0.61479709078781131, 0.13075401904871176, 0.37981082492868867, 0.0054914379420462067, 0.9722712792522924, 0.10588450084895495, 0.28293639530981618, 0.13932416427543792, 0.8162691175344553, 0.11647474753207343], [0.80107664201920548, 0.20248247649386486, 0.45625744950743474, 0.39988601589885031, 0.96099320060778137, 0.54545937376093312, 0.77346765514801885, 0.60033745916566139, 0.76673780616045017, 0.61986771239639415, 0.55335601849418437, 0.21314288822808614, 0.028502443932686305, 0.30166962550118925, 0.82961966905888, 0.82481289035270189, 0.17185151780478103, 0.93234435411101535, 0.53411033057630886, 0.57056106330911138, 0.42958816609157957, 0.080100354967972853, 0.064370537438306452, 0.055949291470661411, 0.96401470191751415, 0.80155748009007877, 0.60415010275469871, 0.97085207160962439, 0.019105096636639951, 0.61889555737133162], [0.16214344426496752, 0.76161487346744849, 0.16011517652313079, 0.51828633282114056, 0.33032067182171754, 0.021227198355766363, 0.022413631068854856, 0.022495390867645737, 0.70560896060332179, 0.70752634985903884, 0.76182570463126653, 0.79386145046851453, 0.76731287617542676, 0.90589279025225022, 0.57553803030172479, 0.28343070246744972, 0.75754546950262192, 0.61559402324932011, 0.24562187645519351, 0.6578439355304434, 0.77307463189642611, 0.95787820812530944, 0.64648935740447222, 0.60764492855778685, 0.84437213673185729, 0.27911526992739066, 0.55310513018866092, 0.66899822326012848, 0.25128546013043707, 0.8537274017378359]] [{"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}] +24 [[0.86425385615782824, 0.066358100833033951, 0.39543090643651169, 0.59218536318562942, 0.48110900493704556, 0.54672210075400984, 0.39615645246905495, 0.82739896249686617, 0.75606385877500171, 0.79976628493560875, 0.43599710181914531, 0.21265034900894442, 0.52263113184775356, 0.19236548383475505, 0.41860217344219808, 0.18425019002161425, 0.53446873982655274, 0.32346216208001077, 0.42438541143312536, 0.51941549065370329, 0.74634061707991017, 0.72185536105555681, 0.75598395945116814, 0.42185191943026112, 0.82809146209746964, 0.52658437019900728, 0.59803328654774446, 0.75944018729458107, 0.036788336180093362, 0.68155347270099154], [0.12978956053927904, 0.14045894491964361, 0.78800163804862389, 0.33717762509488536, 0.34807638433438581, 0.0040335343737816221, 0.13926911736769232, 0.21921104005674263, 0.43322787272996865, 0.79600820727573418, 0.92710612911567991, 0.69855416391783409, 0.22214563130230158, 0.60319718030492675, 0.48349599517001229, 0.175716956628995, 0.24028221735218491, 0.32697757244747416, 0.69861471380805118, 0.17974912271869714, 0.788309382016546, 0.59144522918607545, 0.69361459536769532, 0.27383102205655296, 0.77942418840578775, 0.10186222504665676, 0.52061128583589322, 0.45835164901160952, 0.37969188031543144, 0.095882462397487211], [0.1165894554455984, 0.26718297843255134, 0.016519426058596243, 0.86304999621560063, 0.2037626416426549, 0.16399255562951531, 0.9892990849585559, 0.047322006673414241, 0.95099280696969435, 0.162819274768498, 0.741456529080382, 0.59287675531454576, 0.98919553463801546, 0.82077010074124357, 0.83458350650702517, 0.74522395249153528, 0.80822764643687817, 0.88057807344835726, 0.803030382349052, 0.47764347544315666, 0.65357336260473375, 0.87689494618229913, 0.36127887149775995, 0.077235879005965291, 0.78821620961411176, 0.96257447655947659, 0.88661108988258108, 0.39537570736729455, 0.38840406709448905, 0.059835558593079985], [0.70603605954783233, 0.44411687132383615, 0.40158471478889346, 0.196076538102132, 0.47912984168599948, 0.60310710911834431, 0.41060996308572917, 0.54524550039744923, 0.88739170314543048, 0.69082580008686933, 0.92037633186309153, 0.26305389840345983, 0.91591285271293377, 0.54946816071703031, 0.51139123418079546, 0.67795353983924567, 0.24460048598146211, 0.5945557099282579, 0.98932292233716312, 0.77967992711251888, 0.65389482889087513, 0.98836136111171924, 0.00041398297869532641, 0.23329803456024389, 0.89386231351155709, 0.71049507931569678, 0.57970724634647675, 0.33906863740754378, 0.16574295694864249, 0.81950383561746776], [0.75235434093279485, 0.45708700222488396, 0.22147603980299413, 0.62609737366778362, 0.61784677039262692, 0.42052793579496694, 0.099422791038828, 0.33161922464051874, 0.31282980300128205, 0.37246455036432158, 0.68345045496973167, 0.92464022794933787, 0.870230257841398, 0.27861000074469411, 0.59773739797214109, 0.77405947650069951, 0.27074883691425666, 0.091965652639822038, 0.4311041948983293, 0.40885546456602129, 0.91733844719361912, 0.17524040669527308, 0.9223707838791515, 0.43263822887191083, 0.354110416653039, 0.12303999585211489, 0.53218559794290143, 0.38541093258651915, 0.29818008215695491, 0.56820712194956946], [0.7769670162704112, 0.63545679653393272, 0.95628447781148584, 0.90532634100434928, 0.80704389462041082, 0.061026007412017691, 0.15757447186500184, 0.11929055741033201, 0.97377619097032475, 0.36450404017380189, 0.049804355839040526, 0.067958562329520689, 0.54938279223734587, 0.35100129733698004, 0.97821198683276855, 0.63531598094077735, 0.33839859084052459, 0.39881152881075821, 0.69470307655323593, 0.14518481504059022, 0.65112805654592532, 0.65094434396689316, 0.22952352743387705, 0.30540960095944747, 0.99715832732915066, 0.16494502471385619, 0.598423926622024, 0.73858082435081429, 0.9009398021835876, 0.37247237702704961], [0.015390422642627244, 0.84773946269111344, 0.44163737217753607, 0.7206137108368198, 0.43433730006352433, 0.36413872417076254, 0.47590133986306771, 0.29367321762403931, 0.052725758573360038, 0.42902251283704596, 0.01490172480611518, 0.54717828649678368, 0.87589462844317234, 0.47251253488709066, 0.89591538892436617, 0.025105949287561269, 0.56295872122013069, 0.47007889413003845, 0.713950630161635, 0.076839299167433817, 0.55345071281302682, 0.38070180548709431, 0.960339093433981, 0.14212216028158819, 0.90845296037970791, 0.70071504160433951, 0.45292022746903815, 0.13028686046493498, 0.39078592817489266, 0.10345430826843194]] [{"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}] + +-- !sql -- +25 + +-- !sql_orc_inner -- +0 [[0.075146018909447831, 0.83184654292916682, 0.48344758887617112, 0.73818042587686938, 0.1157914153224413, 0.010906227661216183, 0.14399412358433106, 0.44089849151182114, 0.57355556205718361, 0.25312517135856316, 0.38150647140194149, 0.070397728596659159, 0.86953536344447424, 0.68971873285317431, 0.43295999037537725, 0.6784883106064814, 0.59748020017550263, 0.701551888426747, 0.59172895985772778, 0.34453720788885822, 0.87415544636692688, 0.32542734889457547, 0.15307756621895752, 0.5967639742587556, 0.27981005098527589, 0.2878085040368038, 0.47320794944932354, 0.462573273179657, 0.181991720511385, 0.65965994183508947], [0.30669042061243168, 0.40772045173026605, 0.79942953279686146, 0.79112745662075734, 0.9162031241950418, 0.47977049680735451, 0.70743053959785085, 0.88106846902940883, 0.058859311770482536, 0.5582244050439672, 0.61221349612133946, 0.26398328723774411, 0.845002934668909, 0.7318513641061144, 0.12882949870214055, 0.25568561677314683, 0.63820374314148409, 0.47317928430060152, 0.98579534589233231, 0.18909131775818011, 0.61928522310719925, 0.46076003147362854, 0.54929286054482063, 0.092791921536963917, 0.37257051222485726, 0.61173970164373448, 0.81076330028131915, 0.31761967958249016, 0.86913228888996208, 0.1304288619260594], [0.01395281464511211, 0.30019929214213881, 0.51968481559571245, 0.1529812070388713, 0.91094127854135332, 0.56752084306826045, 0.54885745773319083, 0.87395325967138593, 0.23141383104711721, 0.032857330783252037, 0.80294086357636207, 0.87743211429809975, 0.32541037759757729, 0.610889338006772, 0.24479853782135153, 0.76358175946904172, 0.097015496504703536, 0.18868863101151945, 0.812487590812745, 0.060873224844278262, 0.28492733296189487, 0.029251182023422917, 0.022235039951404745, 0.84492712327496655, 0.86557067014930533, 0.75555932392457192, 0.68176625973315275, 0.44681195608518232, 0.85539631857096821, 0.16569204251912195], [0.34658514841506916, 0.1268049901580286, 0.071234875518197849, 0.26329111773144742, 0.69965586352226594, 0.61403698266195006, 0.47105010114528778, 0.052729894896325624, 0.86403861188530129, 0.96810607785609748, 0.542163311482091, 0.84462116899381379, 0.76838267110247349, 0.30565975588120686, 0.82999691111906371, 0.044838509782801461, 0.076782055123802762, 0.55310856391411212, 0.10761171932456115, 0.65901588915115827, 0.86735027524090058, 0.53604895744114833, 0.87478938101481474, 0.034528135493820389, 0.19797273921449776, 0.396636369100942, 0.84513740655578207, 0.68324497247689842, 0.9330462106026346, 0.54206449863011907], [0.83067163764091867, 0.3808895221813462, 0.72161718918340434, 0.070470082806093348, 0.9655564635473981, 0.1226220247446318, 0.81518155502263967, 0.894996860337646, 0.52416087663426092, 0.92950769116382537, 0.49251073906158571, 0.59241651388338123, 0.68331376414364187, 0.94681786268464219, 0.21921961806975321, 0.33961833290240029, 0.45461966218843741, 0.75265132021366643, 0.31139451010915242, 0.16944322666300871, 0.669921659675672, 0.30415288586162126, 0.795450048218866, 0.176752390721302, 0.341475273039066, 0.48864460278142685, 0.3604857903006139, 0.23628415735788122, 0.93812387557784538, 0.8182746134054657]] [{"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}, {"R9XkCk-Hvbi8ME-BCM-AfxK":95345, "SBBx8N-yORcjko-NaC-neDk":32129, "SFSy0e-UPFNkMS-wtb-kRmh":48412, "W3VXZa-QFzsTrV-1oW-WOzB":99331, "bNnqcr-TYCIXxL-W0x-mfNY":90556, "bRzGcl-D7ccsXC-aIy-lVUy":90081, "xN1wBt-eXa9CsI-s7D-Z0z4":2572}] +1 [[0.36347909454423355, 0.41846626684948152, 0.079114592175514309, 0.66786028640635076, 0.3937097712237253, 0.789970132134358, 0.76656970664847057, 0.33365439648974671, 0.82537818944612018, 0.67177137061591408, 0.27802211322330717, 0.30797446380318649, 0.48919046879622785, 0.026597605734909657, 0.86028930790459013, 0.59862131876428393, 0.63368577900374445, 0.5431679589950682, 0.65036371203785692, 0.44948614963712841, 0.28018625320876811, 0.51570991706205271, 0.025688287032002211, 0.76269923674189588, 0.68635645089977859, 0.16699308447773564, 0.92652113120235846, 0.13746724514453723, 0.28522352481007851, 0.074944393436347845], [0.22280686747520972, 0.52013661953979584, 0.54170574612594191, 0.75841707425706861, 0.22629378640520526, 0.93365330530754753, 0.40014150143354366, 0.27944412692112619, 0.1332052036540543, 0.13447163712493426, 0.38400646685948181, 0.92731394014633517, 0.52359455831396784, 0.33109823808979311, 0.95295239146355981, 0.66406449231411657, 0.040583333692196466, 0.29083957690919393, 0.42249539731446362, 0.031499483050802746, 0.026001816228967467, 0.2599545277374522, 0.60812777394085393, 0.64658816796002283, 0.19852280082713492, 0.12729620943748521, 0.36832215328736362, 0.24511836673200982, 0.42619840669062325, 0.1726601907599643], [0.38984244135290436, 0.47272954855476157, 0.2673810361664567, 0.5584386712736088, 0.88397101557440194, 0.26796130521751726, 0.2323292380097145, 0.4468137216335466, 0.34370717309715526, 0.73370993457568, 0.65236991210468709, 0.25460472165830428, 0.14968718730862707, 0.85149366094614021, 0.63430855897318417, 0.094018303897087208, 0.60652131528355657, 0.25718503570897666, 0.82484947710904366, 0.8899321720123512, 0.94143878487960542, 0.56737682896975206, 0.61310816781385513, 0.012976431924834775, 0.90357683602979455, 0.42230539059768024, 0.95886849725530243, 0.52255921819927231, 0.86494554799831258, 0.49426339882785542], [0.250057791257716, 0.27511637550765422, 0.743044784069812, 0.19969166496450474, 0.4649825842540799, 0.2202471542455382, 0.64030594364852622, 0.74864758117472163, 0.44318905600166236, 0.18103153566971941, 0.826975499454017, 0.46710551238932208, 0.25338160240396523, 0.29470675622636755, 0.58175335891467672, 0.035552091940969643, 0.48765397206038663, 0.07601413130299417, 0.44844392884487205, 0.40159769902524911, 0.80835727577283889, 0.75790953934354222, 0.66707324369196874, 0.644832586228614, 0.22724956640213656, 0.1629642575957917, 0.89437113341594165, 0.99891436279760737, 0.40614244217464945, 0.86366506514462094], [0.84121189081814984, 0.56888116999757854, 0.73407954905913542, 0.800955852695311, 0.1235533940121567, 0.020942877401750382, 0.058153150914470486, 0.84774561366867862, 0.44860858797286907, 0.66750999978632219, 0.72338456817599628, 0.027740412414764992, 0.23943000902995171, 0.92197855672246887, 0.011664804999946043, 0.22983961353479565, 0.93878194064271059, 0.055796222853087718, 0.16057216714992129, 0.294470205491835, 0.52712115242213, 0.55459333471484129, 0.8899536416494972, 0.29061466047068207, 0.23970198873913651, 0.14066987147744947, 0.76747753093944149, 0.65092177705500376, 0.0742669841436645, 0.70588781283143143], [0.80393477414713888, 0.58717195688191814, 0.44347928124587233, 0.98202533050932206, 0.38488675309367737, 0.86950136614463114, 0.24519926868457564, 0.76490786218976625, 0.017328170830664957, 0.62017283159584224, 0.69890177897433126, 0.90413828247774619, 0.39835315178533226, 0.24631301749474188, 0.52797686536944943, 0.072006896893512584, 0.99738354839389343, 0.015919164929985796, 0.45535157681805349, 0.53316887600648588, 0.68433867187319986, 0.067571333797358912, 0.52381605549709642, 0.75093546018670243, 0.93319574155424212, 0.83666838566966917, 0.45263195050630989, 0.68653264186914253, 0.78371105773959437, 0.97963941768735019], [0.098447300967908546, 0.07443874066552536, 0.05364162312387688, 0.88246475334594587, 0.10123641377404269, 0.789920401619719, 0.25962818572797042, 0.75097359949233533, 0.96492943672532139, 0.66414405510781049, 0.32696974165492931, 0.0977680468839287, 0.14759093914880117, 0.747748223744763, 0.44684103813826492, 0.164256312098906, 0.21745266069093849, 0.075518803386006583, 0.37407422008360369, 0.446160635405205, 0.76880054398410858, 0.44281888833985961, 0.049446193515169434, 0.02699795647106451, 0.585661584290046, 0.56868059674765481, 0.67748385157567259, 0.86641471535604575, 0.49143150669862756, 0.29097602701531011], [0.29411933753860242, 0.53888949915174611, 0.42902938687211079, 0.11584074293199953, 0.41809785404179434, 0.17320039917247598, 0.57886051170810582, 0.52653213826344814, 0.59586110883584986, 0.504186608030426, 0.74081473200375825, 0.946162171008541, 0.52169942069082653, 0.015716372598683193, 0.78172190605308511, 0.14425186105926702, 0.94328618052761326, 0.12740691124548464, 0.68113231757547243, 0.17881324017649514, 0.747278356726216, 0.25752393252970207, 0.59598441836341853, 0.67367153556345361, 0.46048722710355883, 0.38062150587929666, 0.4086683973245484, 0.344998612268513, 0.899193991707704, 0.3908662044512391], [0.423188396502708, 0.21153661996401996, 0.85250472959367374, 0.18062171271885152, 0.23057264490443741, 0.849748267119177, 0.49520988865142479, 0.84875310547841587, 0.27737504119357914, 0.38813791036924117, 0.69143405256418478, 0.86424295624850977, 0.34005760124547024, 0.67424626522471809, 0.47243104020173665, 0.32809403833080852, 0.93323877392329047, 0.19332793162241235, 0.75020100667466583, 0.700126356365772, 0.661236755962131, 0.86948025489808578, 0.76317546782863477, 0.48715890459730482, 0.66003400036375681, 0.8379988110295854, 0.84103703975244659, 0.42519956080937527, 0.57073267491522106, 0.64199294836620591], [0.82281096653815, 0.63064910225319115, 0.93576745975954956, 0.076802861309765436, 0.86568542658314207, 0.044158261607361693, 0.51558653015789169, 0.082516973795488169, 0.441588531069972, 0.2456596843788561, 0.78620750347433854, 0.66789690908724575, 0.034369637626316507, 0.38022920182811903, 0.79699996008614349, 0.83827426063719435, 0.422503481741183, 0.501813770347819, 0.64324159000688563, 0.89549206462213859, 0.46055523450336167, 0.11549639161351744, 0.039440817188449273, 0.2797212237176987, 0.957215848262082, 0.078850691907273784, 0.8671130341932527, 0.3309897083284602, 0.58853639051021567, 0.30907140683236778]] [{"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}, {"28Rf95-8tDYrp8-SPm-puPv":54557, "2ZPpmq-00F4M39-VX7-2b4i":60350, "K0nvhU-vembo6C-v6p-JrTs":51500, "KQUchy-P6I93xN-mqy-fCYv":8207, "Qb4MR7-PF4hTCX-V0y-PJN1":49240, "XbfaVk-Pw0hj3X-j6G-O2nF":7967, "i0hp3D-YNwWnHb-l0f-8uvF":2692, "tCBGEj-qDbVqQ8-esM-DLCa":10688, "u1dTei-O06KmEd-Lq1-MGyc":67019, "zylPKa-1L5Q0HZ-WHT-obSS":64271}] +2 [[0.877221380588093, 0.34747559807707384, 0.3675645915876965, 0.52652201116175712, 0.72002970080866924, 0.56105762000298276, 0.0643113918615007, 0.24286385119102094, 0.93193769482862376, 0.73982672018891238, 0.32241219536777865, 0.23776429755650352, 0.34347912299009942, 0.088746649808589928, 0.87220220991248354, 0.39688406891819517, 0.78830357499627624, 0.14570546857506361, 0.76341782137652059, 0.80827256407163572, 0.26071189588204635, 0.0510259473418454, 0.547961177300181, 0.26710245964353851, 0.97103885995839045, 0.57969895488720968, 0.11937708742100628, 0.91930231085860548, 0.56392810599871812, 0.28170721402208376], [0.61324778199729268, 0.25127080800817936, 0.64442747842796988, 0.96350229243099417, 0.48071892107803582, 0.473677028506853, 0.54296545146173514, 0.78764268265190707, 0.96162713582418169, 0.844420885129845, 0.94265218479617574, 0.46779654521973335, 0.70420858480105386, 0.62914880822235431, 0.59098185136086878, 0.3927576640248025, 0.20512469587153759, 0.20728962091476633, 0.82289522124031844, 0.025818600365801303, 0.076549446336385607, 0.29739595747247438, 0.033294705372403754, 0.59479052967517143, 0.88632067547226312, 0.39764710803066761, 0.55448657852034, 0.91315407230686885, 0.46266508583671029, 0.31568913285697842], [0.61765820968986385, 0.15427146464800578, 0.10676508232206716, 0.096558727645921882, 0.20738062263124291, 0.53828476832197369, 0.11444276854086888, 0.74825728217292153, 0.65425987082622938, 0.18527128318725972, 0.37249489131766478, 0.91161251195130255, 0.75324288638432779, 0.83842159286660023, 0.56571961363761714, 0.024384348639726383, 0.04371752648483529, 0.76745149965912385, 0.70743452068006563, 0.58914321983392881, 0.64242759191901944, 0.92515456174278132, 0.28929261484228885, 0.87026831982481456, 0.681358126816789, 0.34894680604821759, 0.33546079855688604, 0.71748411405996537, 0.97012788948265827, 0.87445078128165665], [0.041733925260367566, 0.2416737455994501, 0.97605941389874418, 0.45961152832251628, 0.48191384944148596, 0.75327793868741355, 0.59497952004106869, 0.80553195179579118, 0.071792599319834949, 0.7144088680809102, 0.47193601351918824, 0.504160967135871, 0.09533484173370721, 0.45287812231846114, 0.82339938786838152, 0.64316243674320406, 0.61816382436668382, 0.039658793364443556, 0.81115074183095393, 0.65473896419593769, 0.54021227164917462, 0.0408716239563377, 0.52294022006549823, 0.2078792109912514, 0.37240734879552406, 0.82120578094590679, 0.52613299113540057, 0.11244315855991893, 0.56151756833974131, 0.69775466314298218], [0.77794265759464531, 0.52869506489843954, 0.832795789132383, 0.52156814769376469, 0.39948481109146172, 0.2573388346016795, 0.071811762487268616, 0.19879235626169789, 0.45076590262533889, 0.52280611683199185, 0.72487463470033553, 0.25408864620450688, 0.37985633354816595, 0.45649286316554216, 0.56170383186996564, 0.78609388066071006, 0.26187816992166346, 0.43683491398169838, 0.91451471424145414, 0.068004045368166732, 0.29207182039549995, 0.54955898782761292, 0.18444277163728362, 0.60593412142657888, 0.25208843214463261, 0.6941405987709649, 0.48923874196003714, 0.68352672082392874, 0.82978720032126108, 0.28144550329452567], [0.81542386123156052, 0.29335313189073764, 0.74020035662224692, 0.042727070548302382, 0.93360299793324653, 0.88396984157148062, 0.56896075384837563, 0.20159674922380932, 0.63739098942789407, 0.53606478738120333, 0.13936996071392138, 0.14828055280494779, 0.57441532076986312, 0.021947161583771191, 0.99041799880484671, 0.79941962295795932, 0.052115859607728021, 0.69970558332311894, 0.24544796527828305, 0.026867935186114522, 0.72157093684986384, 0.3466788760269619, 0.74979211522584277, 0.89538643759887049, 0.96933558656034036, 0.84645521797049672, 0.86058860433900985, 0.96182683115498335, 0.1802871475633373, 0.10943452858148861], [0.68312098419182976, 0.55734514265019552, 0.77372978259920266, 0.72905991832340067, 0.8654053265653282, 0.79937892174260883, 0.060518197715768274, 0.65414452632757192, 0.28638236586677668, 0.061568524226932042, 0.520718502726366, 0.59228973973100485, 0.10295558624609502, 0.76749759329807821, 0.059331517881558771, 0.9502113902501228, 0.65724816458046453, 0.66807314128948025, 0.16186224348939016, 0.86386141613295742, 0.68783509231885709, 0.21256131301222791, 0.64202673092804219, 0.274551649625869, 0.13733928386718697, 0.2297398309683113, 0.85064517899945891, 0.066857656163357593, 0.60366346726365294, 0.89832383208563948], [0.79777853353547279, 0.99990042476906427, 0.14326228007519537, 0.90476656178823045, 0.62942294051390479, 0.53567423110177748, 0.70224210910604423, 0.76173251875319026, 0.29558476097885877, 0.69640901780273745, 0.92470874376446854, 0.49198502827412061, 0.048316354019785179, 0.76752627888685965, 0.46766572405140405, 0.27012621650968471, 0.61630088755815371, 0.0035675335801024222, 0.23789929550999545, 0.15857244374998358, 0.51683201753864, 0.491544849801956, 0.15866865827107446, 0.77434351996690831, 0.986542188986288, 0.9794832602050173, 0.17265268574389447, 0.5796893511184591, 0.43899128905126483, 0.7874950360919506], [0.19248748891184131, 0.50227313935811357, 0.19023479676981214, 0.88454634698837709, 0.62136193740762424, 0.7595752288076052, 0.43274884243411038, 0.83614695698989527, 0.88352078678250068, 0.087377402582469954, 0.0835970571561504, 0.60651391907126029, 0.1281582967190894, 0.053632614814570712, 0.18448204193117923, 0.13333082965388832, 0.90006765234999686, 0.77672000701194577, 0.98933216391191359, 0.18404392068255304, 0.1065445359451247, 0.16103139937307487, 0.73933216589315964, 0.24970737514108454, 0.052426004554130934, 0.71302952443856749, 0.395813529660335, 0.13570981881129141, 0.66377857464983958, 0.10736443385741867], [0.27817921459540373, 0.46828475266636571, 0.27563209741345041, 0.93532980863742821, 0.63704408110911637, 0.55152769022944481, 0.52028914957381789, 0.48688539864478508, 0.68203113170081731, 0.88171339793242565, 0.92012223648073721, 0.99266847364726574, 0.47917078643455269, 0.15402659267738084, 0.76240401532691815, 0.48386081515603252, 0.43460876439733354, 0.78138993642706189, 0.71359829678057629, 0.31547488281662284, 0.789335488394516, 0.78589372079599418, 0.41667292013973112, 0.44256667541016215, 0.66390802507538949, 0.42040133140758817, 0.0073779472321962025, 0.73907864535405687, 0.7126434929971982, 0.03122555309154329]] [{"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}, {"IVqBit-QcnX8u5-xAh-NW1M":88873, "Rhp1M7-5icyoZf-zDY-gZNa":66434, "ZDiwGs-ZM4MQma-z0E-Ww5t":21281, "lnQWcp-kExfZvD-UcC-SjPM":74144}] +3 [[0.20833707059306106, 0.26596035255300954, 0.63563007761753176, 0.85541770480854418, 0.40115034778714709, 0.9412613207041074, 0.8295909849011508, 0.48000303149949053, 0.6118879191280987, 0.67759497996881635, 0.92409526089492067, 0.84484713127582289, 0.652624205322621, 0.055773066161040386, 0.51631413723903052, 0.70729443705654438, 0.90360969475785524, 0.17993538649176455, 0.94139391471764133, 0.70606291612943428, 0.26768973741783764, 0.39085624648091366, 0.9210279883272956, 0.99311846433880391, 0.802954368497856, 0.46550326396845243, 0.0757001581009702, 0.54957112369131755, 0.2738844415126529, 0.60878805117310986], [0.62728990184283973, 0.81623919067527806, 0.29192271943186621, 0.26317340679580981, 0.69001936736085923, 0.25487814329356517, 0.67831319141409541, 0.26277926733233115, 0.65026082496178572, 0.72267361580555656, 0.17521545727071219, 0.1408557116335466, 0.26951266783223082, 0.090448619842890654, 0.65551548902295331, 0.46296786659319822, 0.30989043226643787, 0.36413711388433412, 0.53741797850714068, 0.90296456504791944, 0.67776936902221241, 0.67942750404063745, 0.19681939307836271, 0.404778450216702, 0.30875315080402554, 0.931081120042446, 0.7886251903939635, 0.18033239278114288, 0.27593317858514466, 0.47128250887094814], [0.037588568460870109, 0.48230362006032013, 0.89344135803213254, 0.265479526244196, 0.23991225955273732, 0.060222941072004432, 0.9085888388941884, 0.12952336700313094, 0.92876867980231381, 0.95936367473445128, 0.22842185439589902, 0.18851971356307884, 0.40005821862912638, 0.85410120528303912, 0.010315133355070283, 0.49565384044471594, 0.56913928237035361, 0.42090052733362404, 0.36126213323988854, 0.82835462856045317, 0.43492777371213664, 0.35081582212898277, 0.33681082428040743, 0.039512914344140948, 0.76467760533934714, 0.47736268362938872, 0.40813234863403025, 0.98307228537468461, 0.4809184769013668, 0.15285028196831463], [0.074499372612602643, 0.95616144665452452, 0.63144032826133878, 0.44473688654070653, 0.8384895708717538, 0.80403857221409092, 0.753878852608626, 0.94329935114731633, 0.33251914287478235, 0.31947050439179003, 0.90836975329943892, 0.97288318381235639, 0.98785023767030911, 0.334770094038498, 0.64490405338219436, 0.78122672558232886, 0.271004876582543, 0.95256757624221333, 0.69643048148794051, 0.844829374894464, 0.17755750259009717, 0.69049278856170682, 0.24441514793013974, 0.77306035772369908, 0.36956714494655774, 0.9252968102962692, 0.19048288382274348, 0.41968121397406, 0.070449157735003376, 0.91354311441490565], [0.98592838616296719, 0.717681589845808, 0.58987563115403718, 0.83884998075391093, 0.22029973621675158, 0.44180460082325745, 0.14133286950346557, 0.78976749644926436, 0.35092070039420553, 0.57286443894912253, 0.43625278169573989, 0.307723773584615, 0.66713215266162151, 0.18893589054579252, 0.39117484403652214, 0.16107680466600094, 0.826197556551212, 0.84072112896576057, 0.16818384838988032, 0.29916017694370889, 0.07223086525556488, 0.18187684445916408, 0.68390787882160853, 0.51931163370854638, 0.55516957099699, 0.21981032731322125, 0.668649853119051, 0.24022602081373334, 0.0037297677495421455, 0.92843021163308415], [0.33476558020002334, 0.91657503274522634, 0.37444402916704356, 0.12156418893834986, 0.72218937027163455, 0.052687863076487651, 0.99829701876291055, 0.42423218764999182, 0.82904850207115355, 0.47718875815089, 0.17713760986335292, 0.89869044882935056, 0.70659568125556516, 0.52583855705715421, 0.14396083738770316, 0.38313832870421194, 0.800228754573383, 0.046429229429328478, 0.7943165184647224, 0.88320600141404693, 0.95215533763694626, 0.68743174990021461, 0.62051571201569367, 0.41981866841660842, 0.64629923291554581, 0.76119952155523185, 0.83597039995080979, 0.58175785764491117, 0.27233470962312623, 0.082880549961004246], [0.763915402598785, 0.657218831548153, 0.41327372456620282, 0.57779383494785075, 0.18292607853689435, 0.625888164058225, 0.41982013175964361, 0.12287653061623371, 0.47105907557506466, 0.48154289943983908, 0.82474408222750983, 0.041910142955758012, 0.0711075916742423, 0.32489918205847212, 0.47181875155186703, 0.9095855098429988, 0.011376750505389333, 0.41741456059204063, 0.58640092196662919, 0.8360680466607453, 0.95954857266370686, 0.91897475525313188, 0.39852552151738818, 0.20157614630973697, 0.56162024119065057, 0.34587281530080682, 0.34442377316113115, 0.88958726120876841, 0.62356339748335687, 0.88021412583607461], [0.3186631945108318, 0.42763769447110389, 0.030266060887224278, 0.53785831327877565, 0.92664725139005033, 0.086283711321910372, 0.16448326183648088, 0.85923491841736666, 0.19600111300443002, 0.90964324998287294, 0.9879361242134701, 0.22584673473923, 0.74540926288342579, 0.85503893587075874, 0.87736337305276224, 0.12500096603317379, 0.98075951060499322, 0.70461461186199315, 0.7702431698502, 0.68628524171691363, 0.49098765519979914, 0.178399458786202, 0.16806772832277528, 0.59553158066673406, 0.44980501963307784, 0.72614501075152049, 0.069466326846047588, 0.055488626170968169, 0.49453144940436977, 0.80436205715803788], [0.9611374180294403, 0.80714842730182579, 0.24954415519904671, 0.055336060683558896, 0.581463741180078, 0.351651025181417, 0.38939298646089437, 0.69650129322675636, 0.28984900761297294, 0.41991629836589717, 0.89542074527330462, 0.60140508684237648, 0.52422830379075969, 0.4256296286188801, 0.94976717526409438, 0.0075958148105960133, 0.0098690335905130544, 0.10693379112209267, 0.73192354819418159, 0.52211914946793048, 0.094492465401062709, 0.81315668918320072, 0.23554416757507757, 0.11190912777256279, 0.82791096600129344, 0.92513060955043569, 0.10701043689268153, 0.064211879149065121, 0.17878891695628385, 0.71113585040873639], [0.422281520626668, 0.481281620740993, 0.086800859163047916, 0.5152272176059195, 0.2648553689258244, 0.73393968951577226, 0.88537277082707566, 0.550679452709282, 0.94520249303611514, 0.44018979155750848, 0.9983579610541331, 0.72851348343738109, 0.71082424592631865, 0.95793305789668881, 0.713412088649505, 0.88954243817960565, 0.83612616971362785, 0.084929060785173927, 0.26549645105299824, 0.52810237673103511, 0.052586142330613872, 0.64449986463431252, 0.42585253251564936, 0.5966098942061776, 0.38223624766871889, 0.9020762484105751, 0.28495473281167039, 0.26231497878293408, 0.571265999542488, 0.30553626555727265]] [{"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}, {"1dWKNt-58P7J6h-JaT-wv59":23302, "66IGV1-NQEUWJo-OG1-D6zR":3510, "WMrhmR-qfi1kYT-m6E-eLa7":21447, "bk09QY-8f2yXMd-zA8-kB0e":10737, "oHK7XA-GYjR4ep-Uc1-NItS":30626, "pNCFLx-98UFUG1-aKH-8pVv":45897}] +4 [[0.047413350785556418, 0.86115446017354413, 0.52746594712888129, 0.7928220552172901, 0.378383677779879, 0.25869150807878805, 0.881336384872182, 0.43190601429121023, 0.34821786760165918, 0.60939011573389446, 0.57890524957964584, 0.12655679055526281, 0.98326350849592081, 0.4114752881455106, 0.71029502619961726, 0.17406205968209665, 0.4567325446056, 0.90624922448958, 0.375303240937634, 0.0988566852627778, 0.84170079910912943, 0.50945581118823879, 0.45747093315733334, 0.23290494502014036, 0.017642485218647308, 0.41312510078138509, 0.72675036381445246, 0.012354513367785591, 0.4402468598160616, 0.83449032844290816], [0.39116343053711355, 0.83473027751201345, 0.75086596885987145, 0.88173494855696688, 0.50006927291488479, 0.45472645423423164, 0.74979393980696951, 0.030740687327810545, 0.66513612030188252, 0.061693560805254122, 0.0866797027769014, 0.7626717454065175, 0.4098389459056091, 0.37820648027462078, 0.1317269948899964, 0.99914454034022682, 0.084280855716352354, 0.34663840683396707, 0.88293624789722747, 0.58989621584213137, 0.68936292517591768, 0.22945503664309197, 0.43749581574872465, 0.25727945751337289, 0.142803893749753, 0.23898194197162825, 0.5095299444282152, 0.082574869837416909, 0.48923726091599262, 0.13368765830603113], [0.71176397108376144, 0.83102659831568138, 0.30759017801845967, 0.57056343473027549, 0.83526775606625625, 0.15567065261330715, 0.1011751750762262, 0.38697030610078853, 0.15929045737020631, 0.99443972343121434, 0.76940208079065442, 0.40036274221478307, 0.90230102326709427, 0.28069209973022591, 0.35474409858952993, 0.760587387554492, 0.28570063614155516, 0.42422625445729434, 0.13456890649812214, 0.36699452061511451, 0.32327458169065448, 0.88800756582430329, 0.78467158733005515, 0.37138874488462426, 0.3723026411045296, 0.58524615450757, 0.779823373518607, 0.32788461068073715, 0.68634928842648146, 0.7521306371146339], [0.41637640248962271, 0.12202195593296539, 0.69510520269603415, 0.41343528560253817, 0.84294129471744583, 0.414034729724607, 0.50325461327878673, 0.55138287133618125, 0.26738092556915893, 0.96495611869585907, 0.14819369078492006, 0.612559336210704, 0.071840680962485637, 0.87928193325497173, 0.48552689813295713, 0.043340385007249793, 0.91226111171004487, 0.49420333854253506, 0.18162006235943817, 0.13120446073234093, 0.0047967339701571188, 0.3729587915178868, 0.18409015774936321, 0.54747023052045485, 0.70879029312371022, 0.61864436445592785, 0.21396343294667208, 0.63641108936748092, 0.20442861742058904, 0.51486079620275482], [0.27733537518360585, 0.676363120219015, 0.53293319280657259, 0.320864273012757, 0.92971705546238126, 0.050356983104870245, 0.51811452485369769, 0.90142721826526584, 0.0428544573738836, 0.50696841759555344, 0.12034618462716717, 0.75540483270182623, 0.12165847739161262, 0.46649586561365375, 0.5874912834614231, 0.9220413137218596, 0.79798540759164782, 0.63323800492929916, 0.36204009291734607, 0.1903395590560153, 0.98835869725140846, 0.011557952643679426, 0.079694602506116707, 0.014356776372921543, 0.34432467913055254, 0.79158495494945347, 0.15186206684329084, 0.54607769831614772, 0.046986726157523862, 0.46981843504470633], [0.36299633214011884, 0.237555984131546, 0.70136303596186533, 0.00056342957609856192, 0.63345224899351127, 0.53530360458762771, 0.071565919338863693, 0.73604031796679392, 0.59949486370827421, 0.292446915563335, 0.66440120510631473, 0.40617605370137932, 0.94433204913105673, 0.65795753214119135, 0.71683810344194154, 0.41349788841899171, 0.47751519377061613, 0.36730451428206567, 0.9719017582092474, 0.27421494119317591, 0.80477991391046544, 0.55516422586227765, 0.73789191662313558, 0.12348686357132133, 0.056374477453163818, 0.43836286371663513, 0.1174329184732037, 0.82104270304038252, 0.5491508894166196, 0.47601541345675191]] [{"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}, {"V4PuH3-br1j6iN-r0A-7Ep1":38911, "dyfC0y-xyC5nVe-W0I-tl0a":75965}] +5 [[0.81747151151055641, 0.24284083030137293, 0.27264751674096122, 0.68978250796757046, 0.56872724536845309, 0.51449315531515172, 0.053685801305193293, 0.76384965769711222, 0.86742444325808721, 0.17708620392635666, 0.20158720516586492, 0.35341653336535162, 0.37768603540715184, 0.30457216383931474, 0.070651854062912967, 0.66214881379723012, 0.43993158671841415, 0.088312991072720948, 0.55867249478793035, 0.18247298075352136, 0.4324783236970281, 0.1916294092482187, 0.16741542850851077, 0.56457588764789424, 0.84273796215882091, 0.17683757212820073, 0.24317123181410039, 0.37175013805346613, 0.45013937705148421, 0.68156216751123189], [0.038275147650006947, 0.71967438210963253, 0.19575872329921551, 0.67052783878173017, 0.92297321148144107, 0.58725688940704979, 0.7677403280493168, 0.26969568534113764, 0.54673710859106017, 0.40795937269940186, 0.66694309792287165, 0.4015254137978097, 0.99910676743138249, 0.40305759738808855, 0.53384888609503278, 0.68330729752846076, 0.052366942096037872, 0.0020839396039200597, 0.81562081111997442, 0.82492328098686174, 0.38183996124136321, 0.61073649430415244, 0.43130085276655106, 0.06805920930396947, 0.25430407358905949, 0.93282815360227311, 0.95348794460355168, 0.045805414198367478, 0.55102237653575625, 0.030388064313392915], [0.96643913598219777, 0.80974637795659887, 0.14909513809527342, 0.31481731555197712, 0.68338804187325275, 0.67105425283081033, 0.62332386854995214, 0.70203942496922578, 0.609976231087292, 0.932765395217531, 0.95050232816618274, 0.2358752587482944, 0.026044987983461, 0.55233270954276759, 0.39902672805489525, 0.58641130167627742, 0.32463634930216834, 0.26671100580335394, 0.0689430834048953, 0.96391922754647086, 0.79490367485687874, 0.73664658001091754, 0.611027066623565, 0.15392338760358615, 0.26473388433825085, 0.82250608167264661, 0.49821753393246559, 0.57530253643746221, 0.12923497344949031, 0.19089391375943265], [0.84870212801585843, 0.4908506750878866, 0.68284382392931942, 0.0694717693453003, 0.20717950291417464, 0.7585111578883692, 0.34566100135629996, 0.1168072680605905, 0.87929936991626756, 0.80153709496535164, 0.3287209386456148, 0.054743652190665681, 0.38400802437929871, 0.7971206592049872, 0.5182570954050405, 0.92269551568978381, 0.33950404315254978, 0.43557216324552894, 0.87562363040940394, 0.55644053327512477, 0.63073472696779831, 0.98591955477130155, 0.44738663428944325, 0.16839091246792026, 0.019840941266433321, 0.81883763466752157, 0.15919893780352612, 0.71199168812575975, 0.45568425590431305, 0.984105471571594], [0.92742521764681307, 0.049808356144718013, 0.21456711461977784, 0.91732975079343759, 0.26408460294328351, 0.81302921535245887, 0.66148319559160473, 0.8594602912075392, 0.47352995175072543, 0.23952238609369336, 0.85971142307291137, 0.11266382140883935, 0.74338494287235568, 0.20825159525545223, 0.47629431494654628, 0.82741050985376308, 0.9176106984008795, 0.14976862440434702, 0.43191726693314558, 0.618167047260183, 0.50617639300019857, 0.50566455536939536, 0.525932111960858, 0.93887924013456359, 0.53365894032337557, 0.93427584114169937, 0.83074509291650855, 0.814466017425174, 0.15819893655038597, 0.51560571814944312], [0.99451583140917943, 0.22083577599030091, 0.60259112711403184, 0.79533477066397085, 0.71709014310276831, 0.13347515442447822, 0.83089006984831648, 0.62595928621262331, 0.57131650588138227, 0.023449330780670352, 0.57676210288394281, 0.41250991392262581, 0.03037257261800741, 0.36969211568762017, 0.78391719704277651, 0.5509666070406023, 0.031454007797199091, 0.57775284734162557, 0.25430547218931687, 0.55940089321982966, 0.65715564989398234, 0.54770675975763627, 0.316364724110175, 0.64503961016463274, 0.5481003869085902, 0.017723602001872507, 0.623456736372792, 0.22485912496095894, 0.47384844253220948, 0.724829663495571], [0.77173231943064458, 0.22298250425335553, 0.083846009939146149, 0.370686136197253, 0.2870155577093656, 0.80506641413011981, 0.23614909750278357, 0.357746508096742, 0.671900394717986, 0.57802874947599181, 0.45888424787472637, 0.269944692174356, 0.62710351247536422, 0.18288163090708343, 0.037885364575030112, 0.11210455114786011, 0.26521916183394556, 0.63909904273888318, 0.94367156283075737, 0.39355044055251376, 0.12037351897023052, 0.76086173787746914, 0.31190810663154844, 0.24393761429563798, 0.023877341620540671, 0.45868683197901572, 0.47371873255981112, 0.88311051977332933, 0.46382711534504117, 0.76436318669355041]] [{"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}, {"24Ze8N-iOq8Rm4-qDx-Y3au":23063}] +6 [[0.49869616208884693, 0.55144328654490093, 0.55377729558377009, 0.843431335623911, 0.23244546292038359, 0.911558929830448, 0.49086859023363205, 0.36226223621326314, 0.54370221709055011, 0.22534341652795797, 0.49156762889164018, 0.20495232148794884, 0.44319301722603754, 0.050935552057056244, 0.010168390659369475, 0.28971929477372882, 0.05303204383731619, 0.59729213564733585, 0.21790173337764662, 0.31804983072212933, 0.28812816272877129, 0.86135508638390645, 0.24114369881772757, 0.78738240307502283, 0.3430966972097963, 0.43550786859434976, 0.71991441868145023, 0.578146914192481, 0.84066355580305929, 0.19213832146796694], [0.85564343234407592, 0.435748629000409, 0.0063383984157834306, 0.97497754512817558, 0.51659120090499244, 0.10656011036556479, 0.7741723236468423, 0.63098527924958059, 0.89998097667726928, 0.10671112409077776, 0.29970499151180396, 0.201904270559508, 0.71180881535783547, 0.10344387282378553, 0.958744967286855, 0.86120739092823129, 0.81846409850816193, 0.67102632974758081, 0.544415919842615, 0.230012975029124, 0.84870629107123985, 0.74669437511917969, 0.039940966225225027, 0.03464551890238321, 0.28020868327962434, 0.45681855693158624, 0.040925852083370073, 0.67887002644545713, 0.13179281827990319, 0.10946911462007203], [0.62844117902184748, 0.19543041489716906, 0.50064240313976738, 0.0567785789413261, 0.25299261283515107, 0.21437348112399079, 0.83319873320254467, 0.81897839800357608, 0.97950749546020244, 0.039741716490676593, 0.99910444037652235, 0.33171221002814955, 0.26475492906044162, 0.66156987503619047, 0.94947573956155085, 0.42920516730123282, 0.29115567812893106, 0.54672158797487413, 0.98394791662766334, 0.70809530826607958, 0.76472710179465531, 0.34995390933581727, 0.42535278803041332, 0.73020921189786059, 0.99994799062786455, 0.26569954734617685, 0.80655068350784509, 0.11729130209325356, 0.060616121870874728, 0.60140740245691926], [0.51085372559859588, 0.46667209873104032, 0.70331172159417754, 0.29488454810879206, 0.17564138571706123, 0.17408768639372774, 0.87886797401546535, 0.52093346757057268, 0.0681829944815282, 0.18833917156511171, 0.58728964296381669, 0.23668050750855896, 0.073346429920357226, 0.44220738637074974, 0.95284911743829759, 0.50034260806963482, 0.89478567882987881, 0.20501463121214203, 0.48862910025973949, 0.66478168515654512, 0.56436461400100757, 0.88445182996105143, 0.6535694625921481, 0.095046443999420416, 0.37898809690941915, 0.86876624873941677, 0.97011549765063565, 0.88820417071534274, 0.93967534054963653, 0.84513120235817052], [0.16685315259997857, 0.18411578445530574, 0.054625259357372391, 0.65599564157952772, 0.038056853219922693, 0.047409459924232289, 0.886186714706075, 0.1624388351530629, 0.97086334236479177, 0.57375913884478214, 0.87106129432901058, 0.77203061242854454, 0.38082538835436375, 0.82767681722041186, 0.62372997920765516, 0.25842564324427131, 0.61788712739857266, 0.45437767422980679, 0.2450092718991036, 0.25089571181634274, 0.25736250631713464, 0.32488014346213812, 0.20080629618809309, 0.30423085422075191, 0.074287180463463054, 0.37135327044051814, 0.2105566253895309, 0.77291726346425382, 0.41455333998152433, 0.84882620347317028], [0.390492276426594, 0.80822533825039877, 0.12252586808206589, 0.61972576213254782, 0.23380220710688782, 0.6728190952704971, 0.66237735044996815, 0.69930979035333363, 0.87791033264656027, 0.2247237608256264, 0.579168680681511, 0.22291535637732629, 0.50851172005378142, 0.86096523739030018, 0.98574085106079035, 0.20004268872517028, 0.68067039912456417, 0.43092199305891676, 0.3327726515706303, 0.905647022762769, 0.79919710861223037, 0.63040589730639363, 0.32189806516722752, 0.52561234913762434, 0.99368890794445142, 0.16971568150473026, 0.45556641225035943, 0.7212324944164521, 0.14802680622505726, 0.57238281689459436]] [{"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}, {"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}, {"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}, {"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}, {"Bmglh6-1lGdL6G-S8U-qQg2":62674, "GuGCvn-in21qYW-V9C-L5AI":69889, "Jpv3fA-IcI26po-wRz-bQNk":6110}] +7 [[0.3885403004999336, 0.21153443164602725, 0.774279829955582, 0.38300482760961596, 0.43857021473500191, 0.88092208170327913, 0.14180047588093558, 0.71370370179496323, 0.83883388286883254, 0.52333036388707044, 0.64084484459061042, 0.54292690901596208, 0.2899485418945652, 0.24037804499810822, 0.76035846082846215, 0.75291951810081237, 0.15291184942847336, 0.24087319912159233, 0.95160123771273963, 0.58102582686639259, 0.7747184978237498, 0.2966266705076781, 0.55845907896814639, 0.58726020522682, 0.25207289853939252, 0.72422930224614035, 0.44123760571351778, 0.75247122800761068, 0.22624602224686752, 0.36596801119786526], [0.30544365146991681, 0.36826235845698541, 0.70935099628027432, 0.30184474348664203, 0.41991083511232219, 0.52924675607873162, 0.13078875830102454, 0.31470157297007451, 0.36272765763912618, 0.71887748269931828, 0.76234143681609212, 0.4517442455622841, 0.66877785029625469, 0.192008809730144, 0.43136803972705184, 0.378295220765292, 0.76930435066206038, 0.60050793767314969, 0.96378306578507045, 0.60339685076881711, 0.41176004645550046, 0.582651073788503, 0.11197562862643529, 0.071923934156867264, 0.827309320596849, 0.84270138933411942, 0.81755328766189128, 0.064154199247474009, 0.27369710783399281, 0.58398063733426475], [0.534770865479414, 0.82128431301143578, 0.22280240111846039, 0.026910380578020532, 0.99773175162295968, 0.82416787967510252, 0.38504609557665015, 0.1429124849589436, 0.25292505913528374, 0.19259130376680755, 0.96692318614130179, 0.66572305684468613, 0.99838925491476416, 0.24369203421434127, 0.043282304127351789, 0.90358791439818331, 0.27176887532762373, 0.85066321826958913, 0.692576083441532, 0.8689711902765217, 0.311757870009856, 0.87667741818911982, 0.59115788221762566, 0.76682388287465841, 0.98089942476561043, 0.62400143376497863, 0.091801789800795786, 0.93812203724176835, 0.48243888670554946, 0.93456325278650454], [0.421974739751959, 0.94229900671025213, 0.22990290621168552, 0.30577100244369138, 0.52899057090110269, 0.58036018032768011, 0.63530431467954307, 0.14417316810771763, 0.27202784838841287, 0.30988253853869285, 0.27034921502410036, 0.069588268565814126, 0.1150946104721724, 0.68376540532596208, 0.32474870902091857, 0.0036211203839363604, 0.21037512184432305, 0.15651291839602377, 0.53671493136814585, 0.62902201213918874, 0.777994640557613, 0.84499308469062, 0.80294723561407366, 0.69013419907788853, 0.98609031900189914, 0.60503158972073956, 0.488890673891497, 0.51971413521573562, 0.11417793113674113, 0.0908633787608849], [0.054776403617915026, 0.12369787241072328, 0.64315501428246913, 0.30137883300315471, 0.9889290915064517, 0.1330262384199653, 0.0568835289712436, 0.48526509293001729, 0.19705358308354548, 0.94043037096833837, 0.56314599640779583, 0.17655765829312642, 0.93468997826339018, 0.38011830493876342, 0.64587087768975493, 0.56278107574735814, 0.089751608292902185, 0.3795923167201859, 0.3458744156886957, 0.3497718760063474, 0.12303859769372094, 0.11536586193844067, 0.43396828267425513, 0.78044073536410818, 0.25684480092841544, 0.3381378873228047, 0.18813267588093652, 0.48914493113071211, 0.56525338123167468, 0.69802163547245621]] [{"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}, {"3tFgIL-B3QwMs4-pG1-CjAi":32890, "GpfEYF-B4cmXO1-fCQ-69aH":43553, "IYwSnA-wolfZEa-av7-VK4I":34064, "Ki03EO-BK2poe6-YpQ-qG14":47634, "Rl4iZL-bXuq2vS-UcT-QDLs":23257, "bqYHGt-7a2feSo-wia-P81A":62600, "dZ79xf-KyTuXcQ-XJ9-kPoS":99763, "gMmAJ4-FUkEIY6-dF0-RDcG":85940, "itHHpv-vak4pYD-oc5-MhdE":23787}] +8 [[0.23095541823576282, 0.98566778149208711, 0.27275395790578771, 0.75873731923951637, 0.10028729285478444, 0.2559464736048801, 0.28772272775550967, 0.56837815230575883, 0.051161029724324347, 0.13892752805400266, 0.51868974612042618, 0.69435764187724835, 0.883427089389327, 0.69140808027836886, 0.60124850508041028, 0.68588072061700966, 0.91656529819741051, 0.30389467847111196, 0.26237596645547145, 0.994322553771555, 0.10300189155805217, 0.87802175788641978, 0.82823553626372082, 0.4557048288135277, 0.5154890852913977, 0.88063429608427857, 0.041970919105270288, 0.89887521353447475, 0.83260111999349462, 0.37832495444470993], [0.79259573136394, 0.82351192199663692, 0.90831490279849159, 0.15719241890705504, 0.958622295711903, 0.85867720406469639, 0.50480713174259273, 0.81481455458044771, 0.20386284074678773, 0.76049550503284324, 0.79201421791406867, 0.27490611803013865, 0.22787495979701466, 0.036632984058031726, 0.27316516801087443, 0.51985778169966479, 0.669812147016539, 0.89560709006714079, 0.3824115700781312, 0.46232932894166257, 0.59197964606400177, 0.11055649630896514, 0.1188432658874603, 0.78429698284904192, 0.22818461423762693, 0.55995259174833079, 0.73059179393973583, 0.26043067252675067, 0.47200879579964339, 0.66183645224654342], [0.6433883335848436, 0.55240452234891935, 0.40142105043657261, 0.65351541580078243, 0.5398565125880973, 0.71171939682808227, 0.98868574932955233, 0.68525624962903209, 0.7389846669584994, 0.6608439394936273, 0.028312856378552853, 0.99436237109305858, 0.79963624773329067, 0.73595597670586066, 0.23083357981176, 0.0051250055876984613, 0.57762114409318654, 0.969293886968432, 0.3764755350068536, 0.060488911608155616, 0.8027643482331559, 0.16510321190628752, 0.4003497669710695, 0.63287586310982413, 0.61057341297389578, 0.98039065510194423, 0.28955115455991032, 0.28445953299197579, 0.36251888248365316, 0.16017445753448634], [0.7126130919160375, 0.35128785198563062, 0.51960725687424558, 0.79664601467012275, 0.37304608131521355, 0.31100009085510338, 0.29583900154307718, 0.98263505840897492, 0.18122885478367168, 0.060487484660962987, 0.311309281116925, 0.511996339832912, 0.47271028673348947, 0.87769785805392131, 0.489121762045185, 0.34301319661053487, 0.66601862499035114, 0.67088640527331878, 0.99595153052207686, 0.38563924473074773, 0.9493642692809533, 0.2289658273558185, 0.11300575930945467, 0.79937645467057961, 0.81316106541831246, 0.57821139897293561, 0.542295786317452, 0.39581420941202927, 0.50408015647336457, 0.22342733748484733], [0.80254792478197856, 0.92833048997637924, 0.76638976464472475, 0.0057828041350437287, 0.0497442933872575, 0.021183197650795127, 0.55762288585865194, 0.5448774767985145, 0.86919954764256291, 0.24119569815515407, 0.77815444717934368, 0.74919782082951236, 0.32924268898527209, 0.34746769017163615, 0.4380580081300941, 0.67697730533916378, 0.86098788216212119, 0.0680186409940039, 0.099651816192733889, 0.10857475070341394, 0.33434183163508657, 0.84699820289630656, 0.88043221225250012, 0.55183723775192506, 0.03730960181160714, 0.8820738409862019, 0.778632376349308, 0.043514698895639747, 0.13382448760755805, 0.42883288509242412], [0.59243281031957284, 0.032645992891135567, 0.57474226054089972, 0.78751294902881974, 0.24388918575336649, 0.83593660177573625, 0.36730316391640405, 0.6071008721579455, 0.88993255769637969, 0.64050086401134176, 0.57028022339456108, 0.142816790504665, 0.80533523479967428, 0.7959420887743538, 0.37519548480627807, 0.88522730829677443, 0.6563259046040828, 0.18559630396065652, 0.15840084690326972, 0.542707065305499, 0.44857516996876146, 0.79586658598986149, 0.44448309520044926, 0.70369582557789734, 0.12956306215143443, 0.21292640790922279, 0.30836788550791283, 0.63117375092189876, 0.5528781870183529, 0.66767238752755631], [0.86166073395158971, 0.54900483583986093, 0.27078798642076951, 0.422657818643983, 0.11817764368481531, 0.46827444026434994, 0.41145787866633987, 0.74884049713679246, 0.71428137840640171, 0.071473980726529107, 0.71848677537171035, 0.47532944907579111, 0.67155458925877232, 0.42336190921009043, 0.68296539183012028, 0.11191716726813439, 0.63250367844449629, 0.30955693985374222, 0.58205229159767258, 0.84203497329535881, 0.77440249146474527, 0.81739615299087531, 0.15310250653216806, 0.26287142402579955, 0.91204839361554113, 0.36831674122714, 0.57867193038148024, 0.60974447337349325, 0.13711740436411402, 0.786997238570247], [0.43526023902801736, 0.34342003412871569, 0.709969522543866, 0.14972012408805779, 0.26744395120440867, 0.40192158812288914, 0.81916870073738113, 0.092894951355157351, 0.48908987183916475, 0.12247583727700939, 0.25464677116931755, 0.9824067183281503, 0.38462946351831617, 0.82459236475241759, 0.0023543354981858666, 0.84543181435852777, 0.085912095203463212, 0.31187592238899364, 0.26140533956619683, 0.248589747111841, 0.533945656298537, 0.098021720046919447, 0.409136923730135, 0.3697755103116549, 0.59987106576796123, 0.99154589148043515, 0.99516950254571646, 0.3557443968268641, 0.73580687789945753, 0.78415449431440887], [0.9324663430047323, 0.6834970985785358, 0.75373904468502118, 0.17925638417736622, 0.67155609044367992, 0.6507398531968408, 0.589419699622833, 0.7699765353471153, 0.29349554193945915, 0.30917102011811648, 0.19827844328841093, 0.83012803957899606, 0.13781414413314719, 0.12407146805112124, 0.051638249145798953, 0.91695268387956308, 0.79359571913970606, 0.3946536176226626, 0.054067080071233842, 0.80104140373811217, 0.025017081631783511, 0.001849480268928283, 0.086486029175022461, 0.036697582077854118, 0.47707695098657865, 0.3636958586044543, 0.79469300775581486, 0.5651077583098516, 0.81291338957333714, 0.57319998798323313]] [{"5FkFp6-20TVdNJ-z6X-RLzp":83852, "75hGtv-oBTwA4M-2Ct-ZSwW":41293, "A5fAy5-5cSomjQ-CHk-YEd6":54764, "KvreiC-sjRjT9Q-Atq-Htcf":91172, "M1Fg2P-0UAY9Nw-QIp-hdFC":10236, "OEklRm-685Yl2D-j6n-dr1D":21189, "SO06ll-esmpimC-Qpg-pmIP":44277, "T6YWXQ-bEBE0TZ-hN8-eyaz":7230, "Wb4FT1-g9iLWg8-IdK-vRFv":39236, "xyXBbF-WwvYkze-lt2-D8nn":59480}, {"5FkFp6-20TVdNJ-z6X-RLzp":83852, "75hGtv-oBTwA4M-2Ct-ZSwW":41293, "A5fAy5-5cSomjQ-CHk-YEd6":54764, "KvreiC-sjRjT9Q-Atq-Htcf":91172, "M1Fg2P-0UAY9Nw-QIp-hdFC":10236, "OEklRm-685Yl2D-j6n-dr1D":21189, "SO06ll-esmpimC-Qpg-pmIP":44277, "T6YWXQ-bEBE0TZ-hN8-eyaz":7230, "Wb4FT1-g9iLWg8-IdK-vRFv":39236, "xyXBbF-WwvYkze-lt2-D8nn":59480}] +9 [[0.91932272657574032, 0.59023435137531011, 0.30619619687349853, 0.28849927487716132, 0.45352772432874588, 0.24655194318598372, 0.95006171941862294, 0.52677970703670773, 0.19017305345088664, 0.94864324630010666, 0.79828419865008893, 0.72078967469593136, 0.067926954216678492, 0.57003409487722867, 0.77364650920928346, 0.463415345503996, 0.24460360017602911, 0.2250260541917487, 0.23004676483632003, 0.45501787461780463, 0.78118117224476591, 0.61428816933442332, 0.891863056142934, 0.34818051922897408, 0.83063123556143492, 0.69252834123368556, 0.13560817395812519, 0.79510510532443068, 0.88664348168004681, 0.95771328504413045], [0.69874373818096969, 0.874567736968871, 0.55846320012735018, 0.83025145639280284, 0.139780704983406, 0.69741499853336264, 0.1554410984712038, 0.91574874698169118, 0.019301467758550017, 0.2679012119863925, 0.0885282061723287, 0.42639134795440126, 0.82329626889573948, 0.61273838930896174, 0.72583123352064816, 0.67370665435652555, 0.1082145774737725, 0.78055039896280531, 0.87547697087200227, 0.3393974583426943, 0.21138991095574144, 0.56149331836865812, 0.511416421370049, 0.18114005195023197, 0.14100431674477332, 0.68012434569093427, 0.70643304457064726, 0.9366830832581251, 0.93651666514195364, 0.774171710754161], [0.99150129181255653, 0.45227795333678944, 0.87348642788811037, 0.68767853149540392, 0.52177554521125324, 0.4804180335451349, 0.37598299266867341, 0.4790639427416018, 0.49687144356575608, 0.16705861270294253, 0.39527770755294145, 0.072964854773791443, 0.25673477282515633, 0.9799350677780716, 0.31527404020343086, 0.669292303973624, 0.3542761557482802, 0.71729206808426538, 0.15313917842654445, 0.86375650866124642, 0.6920820691577857, 0.2802815672782194, 0.93785779321934271, 0.93118503607398206, 0.50289585706443607, 0.092483474296017221, 0.66993682929975817, 0.87627886169765956, 0.55219527633119492, 0.40644716904099942], [0.19224779968930084, 0.66602167905617926, 0.44607563395436134, 0.24984815151673778, 0.63500124293408378, 0.52377731509013725, 0.33878074983799389, 0.43279913376012558, 0.83710512977529183, 0.28718153344657837, 0.65160630111109141, 0.86304996466124273, 0.29166755410676704, 0.61025387184805036, 0.63853357033999614, 0.39725652950621593, 0.71516140318200072, 0.22472599178525743, 0.76383383133431382, 0.594193907346904, 0.2977766623948277, 0.56640324782238549, 0.89301678290632758, 0.029734136528713484, 0.056743750993667463, 0.81944627175356011, 0.24087892889251306, 0.28360455774393734, 0.48491923893190436, 0.83478001405750635], [0.8690412858125699, 0.3143879807530624, 0.86368302957329113, 0.43840775907703211, 0.28024746140689605, 0.70192113855255378, 0.19220362306223537, 0.36122792086201805, 0.31782739870135956, 0.15519312577051192, 0.67438666743321074, 0.39402211247307994, 0.14611342267009164, 0.75904106446453179, 0.10032882775067842, 0.39960266203884565, 0.5612293671475681, 0.66438538502333488, 0.26085739560093135, 0.498678614463528, 0.26155540218333473, 0.31444253798871424, 0.74174411475153812, 0.086819415945733058, 0.99330425811157685, 0.8432979788636038, 0.058438698496842467, 0.33696559994644326, 0.679028017758292, 0.98341809934972935], [0.11267611252594467, 0.49525626536408163, 0.735506108443002, 0.67512449271450736, 0.37906900325749715, 0.58751364751307922, 0.79234941362711075, 0.5837497881969862, 0.06280490065765032, 0.11954482539675393, 0.9337870714910248, 0.89913927277105921, 0.70882509434381191, 0.97649162617463181, 0.40348607924116964, 0.11735774087685802, 0.38837033923079056, 0.0047794421430829015, 0.43588629732381345, 0.985906816634858, 0.6405364360917295, 0.69478664918530431, 0.50741632217011312, 0.43145284375220139, 0.80448973164335125, 0.97317591139210147, 0.83708835866841724, 0.08301084083433441, 0.24225164255334231, 0.36089954171643812], [0.40835362792990515, 0.95372543768547779, 0.22246810677046691, 0.65621131105023134, 0.13273177180910389, 0.95083499240707581, 0.91233802477738135, 0.86158683681530324, 0.030611128392776688, 0.14610047810278748, 0.00788269323903179, 0.28927602320431578, 0.46574965552351844, 0.15792571513231268, 0.021995939164806355, 0.65228043701672589, 0.53738837435489828, 0.57159124261263161, 0.1452222682307216, 0.47586909995198079, 0.70752308474521752, 0.055469200337252889, 0.8959687105862072, 0.34931670854978925, 0.62076174567336073, 0.71878210235542073, 0.48486308023477453, 0.37836844378098056, 0.8603288318546678, 0.30041317873450657]] [{"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}, {"DDnSFq-kTOvS1d-hdI-DypD":15498, "ERgsif-c12hICh-mhq-TqIk":64230, "FUUlMB-42AUZOb-LqN-zoVe":87326, "FvSmlX-IUFLY07-AKX-dZK6":21883, "GRvRUY-waqB4cJ-PFv-otof":7754, "JKVvNi-Wvpb5Xe-cFp-LI5M":58937, "Ocpn1g-WMu8u0z-Pq9-4kem":66062, "m1LtAB-ZciBlUX-YU4-ICfq":37648}] +10 [[0.68383567150442692, 0.46201676716472917, 0.69037292452019283, 0.99961567844927268, 0.16449502791842407, 0.27555905654926693, 0.15235547393011295, 0.11476967209432909, 0.068832990208895661, 0.8439681913099778, 0.20260637712199403, 0.55626934703922992, 0.089663151057897017, 0.095733193192900523, 0.660586309210131, 0.0042917259839827526, 0.71207824643690554, 0.9046762692928767, 0.41447050633200266, 0.628729432515436, 0.86259477503332715, 0.248023107143737, 0.36614670864997423, 0.6958735657364169, 0.40696963598869185, 0.075121798826113251, 0.25456311913275609, 0.97853411645325172, 0.80824255013008073, 0.17810058673443174], [0.64603158390272153, 0.10121967713471614, 0.71847250735329415, 0.356731165379058, 0.98262937099940317, 0.71860077452209492, 0.27079993071613162, 0.95199119019131861, 0.21363272575388537, 0.47333192227387866, 0.36288119932030871, 0.3862957585564788, 0.19995393666925321, 0.0084096700205120722, 0.27840539364901473, 0.69309446014115528, 0.77784304192133591, 0.9195949516848152, 0.58031752441033624, 0.14450714462928538, 0.095654053215553425, 0.42503007056058273, 0.32753681251604805, 0.99289804875207877, 0.59626499696310642, 0.702031928197267, 0.3163722522576311, 0.030764346091181172, 0.80777977186387029, 0.29514475600195655], [0.22850181471275044, 0.14304212859703569, 0.92642918281217113, 0.73488706149580829, 0.27929612961767969, 0.27690910600322949, 0.028127475592679252, 0.52322939610059083, 0.20343075675237965, 0.28638979750098548, 0.62892945882205342, 0.93969077059497474, 0.1891462472408324, 0.922545823276339, 0.22967311071524921, 0.70104298203932425, 0.67817819696558168, 0.83476038145136355, 0.42638073140338717, 0.36796442767309889, 0.78529438714226529, 0.82460214841238455, 0.14553130124313263, 0.53358546993039135, 0.45953961274178345, 0.75098142578949034, 0.57210590899283487, 0.25172006090664079, 0.40870334677390863, 0.4237864713685503], [0.37834648875454491, 0.86667896201804251, 0.23918893599751745, 0.54291588388164025, 0.93245800557693859, 0.39191208150330159, 0.22537755325365494, 0.24059874555103022, 0.26902258694290893, 0.85132881463937349, 0.61574381048550231, 0.067120569158939269, 0.73668371244547171, 0.54499668824719127, 0.40739434413709208, 0.323550752942801, 0.40369718181013936, 0.0300205590872743, 0.0763644352023094, 0.48979223816377349, 0.4982348807834418, 0.71775553046660068, 0.83792201182023862, 0.12225208300824852, 0.38786659531658962, 0.380109270302761, 0.62929235361605818, 0.0026574673494252909, 0.86143373558741487, 0.53053976702538586]] [{"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}, {"2eZXq1-pP2cZ1l-aj0-nDXF":25266}] +11 [[0.9486624071469375, 0.45111107550701379, 0.34166552478450252, 0.075296801657753654, 0.91907102480777458, 0.96539452643299128, 0.793899676066837, 0.33109932125166563, 0.34468953382531931, 0.087876614564969757, 0.76920181976769408, 0.90334077262058576, 0.77523257013270075, 0.84896407360246118, 0.44543268511641465, 0.5027209394872445, 0.31382080405006485, 0.43233751686839339, 0.72352293042907212, 0.86615406904184356, 0.64213325871723559, 0.14347875353396433, 0.79003470918577223, 0.92109555206168237, 0.38823878701923176, 0.61779623621104063, 0.20912432350075161, 0.8030685046469056, 0.27231757083916741, 0.10614793464422045], [0.27843292774062856, 0.24241184010266748, 0.22817946916718856, 0.25856838581985997, 0.099253921366206921, 0.49740485134206758, 0.61743072325401782, 0.81626474153309325, 0.18339453799005578, 0.57830521794904965, 0.72833649062313122, 0.017454202729087487, 0.4474707850911388, 0.98047073266177376, 0.98287119843405835, 0.32437317492996, 0.129944010995647, 0.52461788295686451, 0.66314490966464945, 0.28679647294840771, 0.507028621782214, 0.18840285874384688, 0.51233838538762455, 0.97166217101277752, 0.28067041343275978, 0.0058936299982919049, 0.26547552170713218, 0.40890621824047146, 0.1651149947341769, 0.21013473321597254], [0.021806464770294354, 0.93036806669055139, 0.50815668860594154, 0.79888869700990017, 0.3832710313159956, 0.11163709142047751, 0.74897816163240882, 0.61477120452056477, 0.45245579393407087, 0.26596650903152874, 0.13656149764490988, 0.84838015009039969, 0.10865552846561521, 0.084890220577363618, 0.41097192008253358, 0.59665080579675456, 0.4620930810883267, 0.071935223273759119, 0.084385164305907034, 0.73855281373393689, 0.024048276959301562, 0.87383023306247787, 0.54955049749285434, 0.92637747191346509, 0.92459158186691193, 0.47786892346950316, 0.49045109171460244, 0.93324591942302337, 0.45981299056272151, 0.51459416239237066], [0.97324418495618648, 0.94610273423450775, 0.35688261492576034, 0.58172726611007042, 0.15513359097833446, 0.64834111146648421, 0.40797434237258068, 0.58827406576912755, 0.76464732747377606, 0.097339616631213488, 0.54302944360088123, 0.60699764487262964, 0.3788637507741861, 0.4695726182280695, 0.98653224967362074, 0.61895504791430322, 0.05144963093819177, 0.71958953448920937, 0.87491512572487817, 0.8355536150768631, 0.029944086675009252, 0.079041175477049919, 0.59890588097142239, 0.99158955617277866, 0.14154231626149105, 0.10806345747775969, 0.17907522216968763, 0.11699039581571935, 0.92370388989253638, 0.70371271117211587], [0.1830380541139367, 0.815489337108797, 0.72923011831845175, 0.068778466464925136, 0.049206839553808668, 0.005218722635876416, 0.84502581303401769, 0.14918322194770783, 0.19469163691311109, 0.40921649324520704, 0.12958054833246369, 0.45931121121612006, 0.95388526276016683, 0.82309819325836364, 0.79627553059513567, 0.531990917669325, 0.21946914817075269, 0.33265444157230306, 0.74794163344070452, 0.45348918319248566, 0.63901913861282122, 0.20589143136901955, 0.30266539176524576, 0.073757086678679529, 0.35808667133109751, 0.14204060735806057, 0.24165927639471507, 0.51830134733955935, 0.59457602515220376, 0.92432035253026523], [0.85708335937531221, 0.57043678044698587, 0.77943336062061308, 0.29374154312744716, 0.081025650298674257, 0.34130924834833665, 0.60510879847052146, 0.038005611374709813, 0.83641860705000326, 0.399960539289254, 0.085948002074791252, 0.87255212007809579, 0.46648980859439093, 0.19683975529959363, 0.65361499231733067, 0.68007073019890651, 0.471414941734445, 0.18154863783748743, 0.38584358893749404, 0.070299787273821246, 0.80916167932615424, 0.48924092525201646, 0.698259525184013, 0.023691995568354773, 0.702083237392583, 0.44997878135301017, 0.21671132167854434, 0.590535359618753, 0.23164082506013417, 0.46148200812397533], [0.091492748516129652, 0.23137527287477988, 0.0007042929168112888, 0.95808876882872662, 0.71503534229241339, 0.40509142681111432, 0.79620111513973379, 0.30737903617658424, 0.66109462093967075, 0.43774135444800855, 0.49445323299059163, 0.47079525512721609, 0.67614915863827807, 0.53828971761954958, 0.41736272890448767, 0.17572702789846673, 0.72267639825779473, 0.066600700590826567, 0.96967667867920715, 0.16390983753282073, 0.39916544846391255, 0.804578548933932, 0.66588237578358622, 0.80462030175393184, 0.54317111349065161, 0.033098943269978864, 0.58985412686534233, 0.991033740524661, 0.46141007221381747, 0.67538188019806966], [0.92119347126658568, 0.774241937045562, 0.024844110055014079, 0.63171460042756566, 0.40927454923779116, 0.6318125104132527, 0.81689188740650209, 0.70094079720243818, 0.57932713231064392, 0.61640375069304609, 0.62935483530108449, 0.1608899906663831, 0.014459657400569315, 0.88483156896456538, 0.86333929053886949, 0.16292922151583822, 0.27334542571303588, 0.26911810285074123, 0.81140107554681806, 0.58507665471243175, 0.8753649267671193, 0.61843441763380536, 0.37233507185204229, 0.88362448326959186, 0.31331802358093097, 0.61448249038386549, 0.481477040992803, 0.78497669809123494, 0.23838971404817155, 0.64274419997173293], [0.42708050146599752, 0.36741459082608019, 0.80287363237196241, 0.088158576401929922, 0.26727651891094217, 0.19676113045319155, 0.64322780686711489, 0.11617030206708978, 0.76232020986560445, 0.70042293190278315, 0.13935738689047505, 0.5987266074187324, 0.593272108609207, 0.89104637011544741, 0.39558943483763209, 0.34968101896290793, 0.1509588632018225, 0.8840321577272362, 0.85462528514033487, 0.24519662046836177, 0.84800218124879922, 0.39241063745601867, 0.57782608808803426, 0.12212345282021764, 0.73190102502729881, 0.52808797734348611, 0.0035074142127339281, 0.57095753330756727, 0.15859652586192896, 0.79023074262598059], [0.42948962705246807, 0.86987231896743833, 0.42364539119463807, 0.079564665799524037, 0.55697042471168445, 0.34244563179264786, 0.54207189996345084, 0.94162120728229248, 0.17985645581872256, 0.36408831151890619, 0.81816464797555122, 0.012472901577568418, 0.50701144347988991, 0.090813391099027085, 0.45292912410740016, 0.81725662068784144, 0.66444845891964588, 0.73886771215140423, 0.19960240089947823, 0.65479187855770815, 0.64031385977594613, 0.90488016722340858, 0.9965823948892869, 0.29085849187844426, 0.30758171972706039, 0.98356442271337219, 0.5823178953786684, 0.57820638983316508, 0.1876966456743363, 0.21364579229422442]] [{"dqI4P2-eGHUQMD-iUg-K3lO":28601}, {"dqI4P2-eGHUQMD-iUg-K3lO":28601}, {"dqI4P2-eGHUQMD-iUg-K3lO":28601}, {"dqI4P2-eGHUQMD-iUg-K3lO":28601}] +12 [[0.058173649168726693, 0.551773880729322, 0.57321059200671853, 0.26016350610151662, 0.1574344683181309, 0.26284302217925104, 0.044477789530217415, 0.28580780548348328, 0.30861084862821653, 0.5164106170109144, 0.93623389406263968, 0.36984161172004326, 0.72020196597094288, 0.44918890774554188, 0.13904099897852906, 0.083955447505741132, 0.41689989968855878, 0.87366108406305443, 0.22879438213432668, 0.7782130160230053, 0.13749845911267267, 0.39231303237113857, 0.58211536755929838, 0.29013593675986893, 0.7333292781314108, 0.41733804468965241, 0.57663713834505836, 0.89423759090005772, 0.36315418582856884, 0.74647945853563924], [0.89916372211944262, 0.733002916741309, 0.74235700341949939, 0.84665312120824643, 0.08722063841225236, 0.23691378833125121, 0.19744189459956407, 0.27921831710125256, 0.56330516328866043, 0.58896950729609243, 0.30125896958372245, 0.76424175446606557, 0.58618322151541435, 0.1182553163727319, 0.42004303910816343, 0.22582649437486824, 0.739868679444697, 0.617461655190474, 0.59026418037096151, 0.786553786294658, 0.80846035563420238, 0.97086009812772645, 0.76145370186620043, 0.0863122908059678, 0.14380066243496348, 0.15209189467401918, 0.59058303416985114, 0.48926623979437611, 0.92295906459730626, 0.6052279437391096], [0.65423668461699869, 0.71244008368193, 0.01383277511423775, 0.25790973386824334, 0.15673500167214272, 0.16037402619911556, 0.519695663290712, 0.93198272841520613, 0.95552626679441033, 0.7825314322389294, 0.45731619234979604, 0.71488502064864878, 0.9995817449293608, 0.583615398581763, 0.43830323432127638, 0.93415587160357361, 0.08219142179075678, 0.011678754407532743, 0.66962883691705744, 0.737780302686587, 0.14647213252109725, 0.32488790350249075, 0.066814955331279235, 0.92468691777499934, 0.25006407427253063, 0.69502779754435284, 0.7820612966609618, 0.59256882589143178, 0.69490531125335209, 0.24431153910164227], [0.98962944948955811, 0.374297738260975, 0.75863858317665533, 0.43178526333686307, 0.34208832001764666, 0.51955405925962161, 0.92611808793758921, 0.20659480368412264, 0.11356466323566272, 0.84489651968533153, 0.51264857914074491, 0.1276077421775077, 0.67993149431525679, 0.62334708489725377, 0.0950028189218185, 0.212360663672004, 0.65090628734044331, 0.813667752959494, 0.22192754216583721, 0.62302830974362988, 0.66531999649787132, 0.63985827021400821, 0.46276229189237172, 0.42308775989381009, 0.58835705399615579, 0.42300450014074809, 0.970959374924912, 0.54557687402412691, 0.6083551175489299, 0.74045930028476559], [0.38870741763807348, 0.80232399284589917, 0.52079320167097831, 0.44143924998052353, 0.099524962634896186, 0.23325072238690336, 0.30838508517969343, 0.78347183091450279, 0.54760447706579385, 0.34582811136821345, 0.97873432868231736, 0.970729557505703, 0.58585154860017485, 0.82699130477396665, 0.50479651935847158, 0.6731390713708304, 0.58897779331315092, 0.1232957971693035, 0.62843277076160053, 0.68425457729428307, 0.3736702979471177, 0.87982375477229968, 0.23362844174912478, 0.73623662437497994, 0.804774224070059, 0.60432651784340641, 0.97348902903874135, 0.6318791253726519, 0.33458933314305928, 0.080315072888647254], [0.84391547910837672, 0.27647854213831713, 0.62179582824263158, 0.99128736514300453, 0.43689913784981449, 0.89644104411968306, 0.80718756166517713, 0.93800766004086467, 0.67766620406218658, 0.71687520170667207, 0.9853067369926839, 0.531730024489774, 0.9524405956536709, 0.63701854235781341, 0.9042360951249655, 0.52848390952712854, 0.66738296273807463, 0.99019908505943177, 0.9502206354404098, 0.033017308554187053, 0.4443027708127768, 0.304133352071531, 0.029453407228199113, 0.64571862744204422, 0.35502955735674491, 0.087714597839108843, 0.046275041525670635, 0.051815453040739423, 0.82386827886413416, 0.062046455889823937], [0.29520029226860212, 0.95937516116430466, 0.086352902291897737, 0.92632845174258649, 0.23359872126024217, 0.48110816792481537, 0.38255147400356415, 0.70330207031082737, 0.78252127072401745, 0.024701760015774288, 0.38446480563474095, 0.94431655818523919, 0.51507825550310771, 0.50738915636822057, 0.70960023103424807, 0.59067657174999366, 0.83493804916787651, 0.24268408496955607, 0.75659047905983012, 0.085783935917387866, 0.52193040296494608, 0.16068901260300694, 0.51376754048241569, 0.41168847507311368, 0.82740203124205147, 0.11811053306794683, 0.43788960765378138, 0.084013766468802542, 0.63888610321054851, 0.93009283103101437], [0.97775883236075445, 0.780825533297329, 0.10181164336608284, 0.41320624127377537, 0.82625651544826206, 0.2038618244292294, 0.48539259300195969, 0.16740471350023078, 0.52436139482023869, 0.97464564181616609, 0.010500704519062376, 0.044492308756156951, 0.61200559359492812, 0.85335258780401124, 0.81923987463184866, 0.25161572307223445, 0.12316236196449593, 0.33490717624747812, 0.25910452019057906, 0.82052625879635155, 0.85634814694410577, 0.72463326290335517, 0.17901770411610129, 0.73683103165756525, 0.51676115277967, 0.1248857527341305, 0.99020943064810729, 0.36818021225953612, 0.52256713143868794, 0.3744419470479331]] [{"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}, {"T7nKb3-Sr24ig6-zra-8BN9":63637, "VX44GK-AVyLVTT-riP-BuIL":34389, "X2nSkD-Gj5Y4qg-aWD-Dmnp":69825}] +13 [[0.089452985160848875, 0.24145592288378026, 0.91129120931446017, 0.13296414413215685, 0.0741821651267025, 0.23983795296183663, 0.83049738874378731, 0.29733913780505616, 0.62421994936966962, 0.37564059775751446, 0.10746463385204752, 0.23260767634931867, 0.550333181036779, 0.62262675190154826, 0.58639853676548848, 0.27594914860221353, 0.8491383732091321, 0.97641469307579987, 0.26025381674438552, 0.1320540318911203, 0.80733606326079155, 0.1765676030816814, 0.077226223567426366, 0.11180611164823462, 0.527222309707794, 0.13201159962834974, 0.11671816080762465, 0.017728349116398645, 0.077731224864236226, 0.3870922745254628], [0.068035503891654914, 0.808913732973737, 0.97233332550918716, 0.65796406593262213, 0.14526949290684088, 0.69866944714792745, 0.82843409907137278, 0.067638423915151069, 0.9282621975187012, 0.26486120764347243, 0.87248841538341915, 0.21290017483542356, 0.13224947388898367, 0.48595207306522492, 0.42086553855363018, 0.60743414336735091, 0.97772077137797453, 0.35498786210128597, 0.58857095049686092, 0.36288683032736, 0.65723054047089846, 0.86928930791198555, 0.2562507999307172, 0.1932206955624638, 0.37734086594927885, 0.7453686034090381, 0.76047539920203078, 0.95760186446429529, 0.59157101446773219, 0.16659626469536482], [0.86311825644054885, 0.99319418045605012, 0.33417390786608225, 0.23540137002400663, 0.30271776982914023, 0.43161590781759096, 0.014755972146636931, 0.57229670701794766, 0.63402894075590788, 0.541447993112259, 0.64020101593791612, 0.68077787189153771, 0.12154228327341365, 0.894065483524574, 0.19305496357649654, 0.280153192528405, 0.16856844088356304, 0.81119683423884281, 0.8144732923252721, 0.79761895605849364, 0.13555608329861424, 0.11999834170113721, 0.51502785438362519, 0.60640289882289367, 0.429632169167375, 0.71506379868778458, 0.36811611586629933, 0.3985557291220756, 0.43192294683630672, 0.80667257624430433], [0.52841009585100163, 0.2889794454105592, 0.17003531126427662, 0.020711653678343711, 0.96172822381486756, 0.86153772056550437, 0.640147392419355, 0.7080904141267963, 0.47290862871576045, 0.38860266820952272, 0.33647885509426712, 0.92009958613738885, 0.47470717663755735, 0.88214931515268424, 0.85548027568905627, 0.51104828626665422, 0.95343968934960244, 0.46751711467212509, 0.40790267135718083, 0.74064677678298318, 0.25603232740526682, 0.8102158501920429, 0.69397727756498884, 0.56598410732685933, 0.64833980563323268, 0.97166088072968471, 0.875257157376079, 0.53658389408157547, 0.076036570052034014, 0.12148041957715749], [0.31124150067684797, 0.83590332137394174, 0.1970809904391656, 0.99204578862354, 0.31972460212453935, 0.54877384335302026, 0.54869809979547113, 0.82513271335913407, 0.4668813337781853, 0.67666470780574151, 0.66202193412406807, 0.83113028094347308, 0.79642137441576089, 0.77128300571481312, 0.043983422738506861, 0.85797125513331607, 0.96671489659887289, 0.31190731821280282, 0.2087256749537576, 0.049966141698995381, 0.48061247209036706, 0.11298416057873084, 0.35977288519973816, 0.76598009032380687, 0.0647524707253706, 0.11589981425504337, 0.89937576492886318, 0.92190060600910562, 0.78346949692792323, 0.89247252755694129], [0.6729000417484704, 0.22410129509896448, 0.65698769997439288, 0.067593346250181763, 0.74382162600110413, 0.46861460415755185, 0.39440315353479516, 0.65457592032509448, 0.53883590152375227, 0.19498463467283267, 0.34902950974143732, 0.989159399017759, 0.40278467456393774, 0.504278917666046, 0.95700205963968377, 0.54674042594771843, 0.959549647484965, 0.77749537564721849, 0.72960522331596767, 0.43229366271885006, 0.18775615507669219, 0.36360920919929984, 0.33560843705823962, 0.74357286298375325, 0.789076652106435, 0.045761707809252261, 0.43196684699645449, 0.32556731798239746, 0.25211837246464508, 0.23960111535846207], [0.67956232919706072, 0.6578212307118686, 0.99632542141653246, 0.42954308551294162, 0.81523123296506561, 0.7461068601134887, 0.11802010769196847, 0.30672690806921121, 0.74214574137292388, 0.92607423143004786, 0.55465452186172359, 0.4367003554636909, 0.34768557766514741, 0.42888390382857389, 0.93353780929245667, 0.055374772060051813, 0.16125743339614018, 0.40746153265449636, 0.97775071211410625, 0.768549465322035, 0.024089672446446331, 0.87689601135655926, 0.10465356132179215, 0.51331200297139212, 0.65885017047581673, 0.68363566490784511, 0.77133968448018464, 0.12887953798917218, 0.39900571602069956, 0.18551993319670812], [0.1965813236587205, 0.27995331515198174, 0.36616116934649634, 0.79831673763396793, 0.32026051136954348, 0.27741681541790353, 0.068187500373316445, 0.94431835652634866, 0.24897252707527395, 0.54921574994780353, 0.731767597828022, 0.22468172574615042, 0.980435164228272, 0.12851568149465453, 0.6800596977173915, 0.82958175171499771, 0.89953218372728883, 0.54674115747316121, 0.719497693354742, 0.2988789803547095, 0.39526428235218791, 0.945857902757741, 0.80931887387125723, 0.75088600215596535, 0.96057710957541842, 0.057267466997572791, 0.25322924692719695, 0.60794948521280023, 0.3059803131155493, 0.93394146786090471], [0.27468723131730022, 0.66494772824701676, 0.34306712881524137, 0.9679304267433696, 0.11560663642059088, 0.070436369243278563, 0.3754898045273346, 0.77714023160250878, 0.73285055086105311, 0.42005795347825214, 0.47264075996912058, 0.0032765290387108958, 0.75770732868874358, 0.69796329516510514, 0.79523526384562149, 0.92711570717651548, 0.63061128299422486, 0.24437589679891059, 0.59218643998292775, 0.077768734565688, 0.72668953358672506, 0.27606355495712431, 0.69460860903130373, 0.68936184738718442, 0.84728847394751872, 0.74136351686951885, 0.59548080833203021, 0.941566017122033, 0.668451996406095, 0.45254091686079589]] [{"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}, {"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}, {"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}, {"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}, {"108d7t-wRLsVU2-OYR-2kT3":43147, "WxUAa6-vrl2vtK-FdE-pQ3m":9864, "mBdOgq-zs5yNcp-SOp-9WJS":86381, "rfEMCW-qYS2HdN-3IX-n1Te":35090}] +14 [[0.63221100020486043, 0.66419100187251068, 0.54223407509079147, 0.55614475388159912, 0.62567698285273765, 0.14408921908249051, 0.56632600009444878, 0.98746156764645143, 0.3248061233757219, 0.40763405487584814, 0.958919854781932, 0.88413941373312155, 0.86763180024487319, 0.060569617157883893, 0.15713202624220213, 0.18452066167354397, 0.58431245652517017, 0.94655573283297623, 0.22745176644958731, 0.4239152598710878, 0.15127906689835824, 0.63753249863689077, 0.874850393117698, 0.69835449664946836, 0.18242563808798706, 0.6441862750350752, 0.52200582984756139, 0.48505789848958369, 0.96123103448756142, 0.20787559715793502], [0.97942435183974741, 0.20489809225267197, 0.59680130710888124, 0.8410398345460991, 0.82196923738474537, 0.35843671074313821, 0.74717374745210341, 0.62002720673843326, 0.96328146253169766, 0.044644080282096654, 0.19482533021478765, 0.60244335516218339, 0.076804729483281986, 0.68020430457242875, 0.98217954164251975, 0.64053484820396189, 0.4506037084964003, 0.777346360008795, 0.51900758863901553, 0.29278695950197908, 0.58289035368090414, 0.7094310182273702, 0.058971082876968839, 0.49782860565431419, 0.32615076025601986, 0.49606224585597158, 0.20915532692515604, 0.16790152994652041, 0.66999037534339312, 0.77328112298770391], [0.68429332887915306, 0.88356476439363463, 0.3162928073857868, 0.27943912347086008, 0.34648279882084865, 0.42696998743638848, 0.045013901339934881, 0.3836366908530775, 0.681549712385942, 0.9779563391514231, 0.49490885825544351, 0.41115217041893937, 0.25481310023060844, 0.41537874705690103, 0.39071729002112232, 0.77907543942660684, 0.36185866812454248, 0.31549672751862756, 0.43680733538941863, 0.24539894149706287, 0.044028425548914396, 0.9199151029104512, 0.067006692785546829, 0.89610889974352093, 0.87368692810757709, 0.55748260803903915, 0.99132473989960879, 0.84735756116052063, 0.43092527054211072, 0.74771717766891688], [0.012496927234595723, 0.023587092609008931, 0.1912155365825613, 0.48774247434239959, 0.898758683710582, 0.78751444972047635, 0.62263064673696733, 0.030382539482552096, 0.1939829452450117, 0.644175813327713, 0.65159521024981593, 0.7979769024076947, 0.72229189004552707, 0.71876791602486645, 0.29773545050484229, 0.94464174165867276, 0.20148016301405836, 0.21681314997324119, 0.20124700253588712, 0.47911694979362418, 0.81806296170386172, 0.61521392325881541, 0.97030155098819126, 0.791003163458028, 0.83648984935162052, 0.90405107430982057, 0.76136466656851121, 0.8851067913753019, 0.31604042074127836, 0.8015201743131376], [0.27189847928804567, 0.62177507214604433, 0.7911459361316322, 0.49818855915111404, 0.37277493641051818, 0.78551710218288517, 0.3343563182720406, 0.15179492021452956, 0.93682635011443183, 0.859871590120638, 0.765899380121936, 0.050102605160703195, 0.030311994538840259, 0.97339497150622256, 0.89975538901308361, 0.0868565561096335, 0.525680861065447, 0.59980855430900726, 0.62339634324741777, 0.95468304228339362, 0.36686218866382725, 0.24475716433502259, 0.29681642926934559, 0.2488103710436762, 0.17241700818754735, 0.53229543756504016, 0.42620650677003546, 0.49677774958216614, 0.5972356607558067, 0.81613256645269772], [0.81860158240723835, 0.9818387171915387, 0.89006744493915, 0.68382989295445462, 0.45605773428652341, 0.11155431513212255, 0.38202064590028573, 0.49656013006448474, 0.14139039678019849, 0.99811071891280012, 0.22027199655023322, 0.4127008447901459, 0.5622333766756471, 0.863184459371285, 0.5215771560027973, 0.18044881937967039, 0.79615434760381065, 0.85771803315777351, 0.26177436865874637, 0.35280659044731288, 0.43678829233475958, 0.89575747452759313, 0.64032585811885179, 0.47770541854310622, 0.45942859088689325, 0.14154249102343097, 0.36304240575432067, 0.246694157195802, 0.6642400919043715, 0.86032889176325245]] [{"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}, {"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}, {"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}, {"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}, {"QJQqKV-rwvMP2Y-VRv-hjWz":49596, "YhTFoe-F9E7Kxk-2Rd-0ink":34130, "Z8y9Iw-qpQasYi-N1T-h9S4":22585, "bIxpa4-zxivDg0-YUE-awNf":70930, "uYyqDF-EdY0HCc-RXU-erUy":39919, "yGj2Zj-MD0MYzL-lCj-jXFc":75180}] +15 [[0.63885556051479053, 0.74535962205844652, 0.67486103922792462, 0.17793005475060286, 0.59375151527306469, 0.12581600279247906, 0.28116095067089286, 0.51338990857530142, 0.72721199812358517, 0.694769409584071, 0.98388721809677948, 0.61233900076897285, 0.81083138973737268, 0.3419476586429343, 0.28120160344629874, 0.38455139677956807, 0.63223240609624642, 0.842531471683803, 0.24496461148998405, 0.035380584800178805, 0.50866799829363474, 0.54379505313084409, 0.85556767354120733, 0.33144679801048915, 0.78137914106922657, 0.76596973992721984, 0.74081547643732781, 0.65923695212449118, 0.42172032790049663, 0.064211490243435421], [0.75699886622210344, 0.5220331221341511, 0.78338512592403753, 0.8471864082908469, 0.95674466603689856, 0.7973440734717665, 0.094634284994410067, 0.10267088701853488, 0.7509241367276358, 0.16002133411119368, 0.50024953587756893, 0.14169598743504008, 0.49084634877333633, 0.5654997150291452, 0.33500607141996419, 0.34014830466876944, 0.27215910754946981, 0.58310971747006446, 0.83877710773650582, 0.83037261622607617, 0.49332460006838774, 0.67116721790384959, 0.707954929625355, 0.97520415825326523, 0.58923256061975848, 0.19355345037687621, 0.71869185764408927, 0.24299150384167811, 0.40445899505003491, 0.091756371092565958], [0.1545361188196871, 0.79455330792326406, 0.87087389521336855, 0.64685196333805473, 0.92435712886100352, 0.69164585636065, 0.01074648426337399, 0.69266244221676165, 0.06602541964776687, 0.14140008948175298, 0.79919225400539762, 0.97433514465777638, 0.79713249413831155, 0.6023205560556194, 0.088750718562418451, 0.085844970450387414, 0.3961064984656586, 0.89482092686777848, 0.62407890261831622, 0.56542982112535423, 0.97539941187248147, 0.682903820629692, 0.15619020176355103, 0.27972719963480197, 0.64299249582788753, 0.82347542784335159, 0.80336398978236179, 0.18538754280166847, 0.63928633196377971, 0.11591811806659669], [0.0997964745527028, 0.92119759954517444, 0.78351264127833953, 0.92692737452561513, 0.62325886366158767, 0.45425522603966917, 0.96712496430355754, 0.01425157178570613, 0.699473566665951, 0.18049471838165354, 0.49119586255277015, 0.62785683041714513, 0.44361551105344876, 0.46286945264004964, 0.76066865168086784, 0.62817103491396409, 0.31828986868155063, 0.57975658348155834, 0.22038175170071661, 0.9284446191148652, 0.46253376602310403, 0.45165624004743921, 0.44610419495548748, 0.04189826953178, 0.27783392121341222, 0.3160158113694943, 0.98839108720272462, 0.89421037399870451, 0.14484836877038021, 0.72801699853182678], [0.90788843736623326, 0.3480302681438916, 0.68846514618416932, 0.8794219645166137, 0.937786296942425, 0.078585525839186987, 0.709026260520254, 0.24808230399604836, 0.86244072485364875, 0.53808513088051269, 0.75399078699985234, 0.017472025353283072, 0.010753755635567797, 0.75150233076753425, 0.31700597916796125, 0.53861456743250125, 0.72872272062930055, 0.5974546027349934, 0.82030059639437147, 0.34603042804882711, 0.31763878586328764, 0.045280268294567105, 0.86151029397823009, 0.88484081988798369, 0.2027364748039272, 0.54304467689273006, 0.68748995016995773, 0.15792003269267263, 0.031664831628613888, 0.38109118749346982], [0.49536293824136624, 0.95999702923594865, 0.9929723986700324, 0.39188543551277577, 0.95500329068021561, 0.22986505770747068, 0.89186781514453017, 0.78928403184380391, 0.2973412412873605, 0.32236501115288052, 0.42383802209942234, 0.29899125834540785, 0.41932861553483924, 0.678567258270969, 0.64627728733363, 0.98685100513957125, 0.71909552621132655, 0.45558323883937268, 0.86409913149721629, 0.35275295052671307, 0.79377474400929038, 0.16669680616147065, 0.017375256087800439, 0.32531583759536187, 0.70766678413879758, 0.27801293425758078, 0.18736418912457153, 0.76614268896544879, 0.26986200958326612, 0.0095788086263627337], [0.021864101602305785, 0.97232590279467757, 0.64813354353825148, 0.2969405740622757, 0.72164359397874822, 0.45201336101498291, 0.68985825265228562, 0.9833569291391302, 0.72677493221928158, 0.94022885206504458, 0.5209196018715877, 0.83339788752336719, 0.39501482660378773, 0.22077327680195624, 0.34999097790306055, 0.88541548722859509, 0.77394736541191489, 0.82491585220013641, 0.91829342633121691, 0.99338103841066627, 0.034424881558236975, 0.088717627738795213, 0.70386703630691827, 0.14784074773715616, 0.2549122885099987, 0.36821168333158616, 0.81459092820738643, 0.31124986697308243, 0.024660992660288317, 0.43529806747249]] [{"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}, {"66wOPm-CubqZ3D-6q8-PAQ6":30981, "CEYhza-IvfrVKQ-dFy-hArn":95705, "OKQ0si-E4Ry5Er-MYn-d3EK":52824}] +16 [[0.88314255956027632, 0.79571874738813719, 0.97614372903888258, 0.37526204580470068, 0.494410856072649, 0.18558609496466283, 0.53527235841937837, 0.5038418765180156, 0.11777341444796707, 0.52338620845911255, 0.43109683101318719, 0.580587279361151, 0.75325940279621928, 0.135287423037333, 0.085888492946933792, 0.073282267922493816, 0.68090685035952847, 0.41301815871956771, 0.027978199771674928, 0.90885506087912049, 0.90747424628037965, 0.737383184415786, 0.0005222574143753933, 0.77224015940942659, 0.57151532354385737, 0.44059565505221232, 0.51357166143958088, 0.80802396061704085, 0.76619889800477536, 0.058065299999449094], [0.79736035082604262, 0.57170591099298806, 0.58788682814914739, 0.22672937891252976, 0.7914873649413845, 0.6369779410242965, 0.47336764687500288, 0.84970745641833911, 0.88767383982335168, 0.48124507100827008, 0.049774725006147658, 0.56600718160898111, 0.63303540777125555, 0.14368044361516696, 0.7886017689269621, 0.13920296277699651, 0.496316977686446, 0.67842809921741043, 0.7288606060018582, 0.11045510134680092, 0.10129527308124719, 0.61013771286010232, 0.41430928333090156, 0.94893201199322752, 0.039562565620639978, 0.43983494730554462, 0.97581133560278754, 0.97664372607972172, 0.48916215352245929, 0.74620236780263727], [0.98226965143667122, 0.54488976070263673, 0.38605475481159079, 0.58066546463404756, 0.28221163832961094, 0.29597057294519025, 0.8221518817496194, 0.66424863816689206, 0.56298456600724489, 0.85650595997180523, 0.67666108552809734, 0.35998719689027392, 0.76565341099525874, 0.9776501571617342, 0.22674505873656825, 0.51675028311454718, 0.81107601131518858, 0.96268810958463935, 0.62900441588730016, 0.058651448643713411, 0.31626295714491659, 0.20394274153900971, 0.71505233078109265, 0.62879273607611075, 0.11001310632758698, 0.78861874148487865, 0.50245643970363474, 0.68690918489834973, 0.39294759818145153, 0.42715305949957683], [0.18658144359949791, 0.20755056087058976, 0.32944871048842883, 0.40783841025851342, 0.14937384091708428, 0.5287577693110117, 0.7076425829432631, 0.88793045724835085, 0.69788768911323484, 0.96076136765761311, 0.058447103158285163, 0.59414570404425715, 0.82835838914130988, 0.48005007691260071, 0.285839981186071, 0.92440142761381, 0.30088905133080712, 0.20622774332269556, 0.42788409619989332, 0.51784519065415424, 0.70532216538823944, 0.387396897045597, 0.64168031137228632, 0.59166594394400451, 0.00062679520934627586, 0.30437811387169045, 0.885300779010772, 0.73087491776626845, 0.21813925976259341, 0.84489411220066457], [0.05588855096657297, 0.36118888695641416, 0.63325354250655719, 0.49591804128105843, 0.53549203942464207, 0.83534452353256772, 0.7306650780288646, 0.15570590374188487, 0.19483567452202011, 0.91961756314276555, 0.715029317737101, 0.47660302884219086, 0.91011124604729265, 0.48001521138549874, 0.15190472550410306, 0.41553503061918196, 0.55900671955501835, 0.336556987607466, 0.56310727296799234, 0.29116413628073912, 0.27957234720737789, 0.57925281674972173, 0.44611137421102087, 0.92501163322314728, 0.49915712899028986, 0.12083375857741363, 0.23889025256991081, 0.37841788847112279, 0.87053049641630131, 0.89898546531165957], [0.20959617860231794, 0.38803179474534166, 0.78843812430500582, 0.93477996549384124, 0.11473927335965362, 0.84959117553013808, 0.60222252657289732, 0.40419661096907389, 0.74003083205525944, 0.60307693221967051, 0.98877261228428026, 0.98606842731052913, 0.11481549103256239, 0.33189300808944933, 0.20494870318203384, 0.01248519898248468, 0.27402405377592076, 0.60454554018055429, 0.099762863804714219, 0.36663042723756512, 0.93083654205563526, 0.95936402669656362, 0.83397660943626739, 0.64822383930819261, 0.31875323987316473, 0.46112830695942686, 0.22841120678795712, 0.85818724800556689, 0.76352874996377129, 0.34303502564704158], [0.54214880949651245, 0.74184637055627034, 0.25343254589002484, 0.18825786578496106, 0.267934800377606, 0.9739266068343182, 0.39055650991295243, 0.60242587448528107, 0.22447532908537016, 0.56879622854450662, 0.62639870160778222, 0.48166472969977736, 0.8239561571645031, 0.25585556985221714, 0.61170967733113712, 0.790416060223728, 0.8979922495517918, 0.072666426196961731, 0.82359562424473776, 0.229070173270136, 0.87635011543530483, 0.097360694973200324, 0.37213083331421382, 0.29351794294520517, 0.4301676109829129, 0.067062998342373947, 0.0342393022283175, 0.0070477665635586551, 0.91502905492925046, 0.61050730335250647], [0.11436318555256575, 0.039584178806211123, 0.07913603228044408, 0.96072259150120543, 0.68944412742122341, 0.020084495734236096, 0.53794719963903626, 0.30760839745994406, 0.16000044844437145, 0.59498051486459136, 0.45904313201874392, 0.40041582124856856, 0.79959181270414115, 0.188872870217362, 0.74960539135879034, 0.616595222749575, 0.17172021368983215, 0.48325501486866029, 0.793781046243252, 0.47616745722143183, 0.86622431866487415, 0.34486534575302141, 0.44833775120975983, 0.499719923909254, 0.9964817303393575, 0.30733527060846544, 0.572396328192084, 0.38332488699399181, 0.29424187414235503, 0.73812197895303577]] [{"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}, {"6OqK43-gDng8jJ-U0E-adOa":76596, "BFOOy7-E8O0szT-tsp-NC3p":67845, "bqWPgK-q1HtLVT-jKQ-Cgv4":38663, "jrFjPk-Q3Ea6UD-Azz-ENPF":14772, "ttisAh-20hKzJE-Crh-Gn5u":46542}] +17 [[0.88177831777895144, 0.38759617346917141, 0.11526365564396113, 0.72043868520836585, 0.61138479638383181, 0.9620115658850783, 0.0084347806470101983, 0.484256749062121, 0.095957799189622328, 0.50705669354063065, 0.28536282230644472, 0.031494876720769782, 0.29391281021556992, 0.88795459111977881, 0.091578864840146212, 0.28793303966902617, 0.96837305368696081, 0.920353399564873, 0.90675818334837543, 0.046597266979181429, 0.26320971953020911, 0.95490229999020537, 0.37673278661768461, 0.71874262887924723, 0.0099980594643507281, 0.35302903108805439, 0.19876192272915494, 0.017252063855499378, 0.92351298803199611, 0.91999049193889715], [0.13809390704355751, 0.53158813929148119, 0.78576999793704538, 0.81146116698775883, 0.02436080949620989, 0.55675514012742022, 0.56725041390237241, 0.036371636799054308, 0.753104616999493, 0.59829877518963637, 0.1888326993063203, 0.44140279319319609, 0.90074210462674176, 0.69119169229918165, 0.28908445506143088, 0.98148816009188977, 0.550533064323865, 0.042522859563579707, 0.92895229961475789, 0.1558710309955974, 0.32262425718187127, 0.53622932561168024, 0.6395197674367602, 0.7207092620205251, 0.40299954931027404, 0.32267933480404032, 0.59838662151817623, 0.17691506366546927, 0.57417432761310683, 0.056735797605043659], [0.0063546358628056021, 0.075863069698958951, 0.8647300514312749, 0.34499966991365083, 0.92848931788784672, 0.88579503297555617, 0.45529291749578282, 0.61625289474060441, 0.96225815815090165, 0.01819480829283493, 0.90531795154660588, 0.93869126317352836, 0.10033025321560973, 0.626102548465399, 0.59804811884630271, 0.27335727956458322, 0.0667540482176242, 0.95528257454369825, 0.54027544971423647, 0.27525386506376315, 0.22261307997914581, 0.35435443948531042, 0.2990395230681554, 0.031940043690750208, 0.860892464326198, 0.53144195714701348, 0.90915563110488129, 0.0639797751010891, 0.34017158179756124, 0.28259095164919745], [0.043473795232956691, 0.50243572212798515, 0.042891138110540905, 0.711150528662088, 0.602026687588878, 0.60566970247055274, 0.58632585229325251, 0.90750954834747455, 0.14673287609322705, 0.6810324371747799, 0.063026755951264612, 0.16502372909565521, 0.28507946356291181, 0.036594902553426012, 0.342379456807115, 0.0706655264000805, 0.18921598225703207, 0.80389424778156926, 0.62605842716675786, 0.22787315838668198, 0.92614931949542545, 0.3447374866787587, 0.51753914507196153, 0.94427097157928785, 0.17023698101125606, 0.48342676910590665, 0.83946384041138467, 0.94837882028589626, 0.17093423426683718, 0.99033946504674952], [0.31253807572669878, 0.087122868201550241, 0.55675812395912094, 0.58361781616420261, 0.9957698309852121, 0.99990614859308313, 0.6706894766817314, 0.21279232329378817, 0.0802438749069706, 0.96519204498026656, 0.652741350880272, 0.54148365201730608, 0.22725838513039653, 0.47199949980667832, 0.41218089388901558, 0.3558981769350833, 0.1465328697744247, 0.58140392780090067, 0.96489864716794194, 0.091652691251193907, 0.52132931973090446, 0.812083069808577, 0.35067767607843059, 0.79353367085537774, 0.70471169256044064, 0.10914247613653161, 0.49062253959164581, 0.60111758936733384, 0.83480108817378806, 0.43690056846139813], [0.89676583307612245, 0.2877480778656013, 0.14676743952428584, 0.67400841532630218, 0.10387799767308004, 0.47118705475392963, 0.29543069001837319, 0.78322495991841046, 0.057763845704690953, 0.86132654742912185, 0.44472773686372868, 0.6783632978970372, 0.46483704149051419, 0.31366850011712522, 0.72893412778099931, 0.81736952056697509, 0.28452154126590878, 0.51685015974112392, 0.44541193523097733, 0.31319026028133579, 0.12903718163832067, 0.85831308611021484, 0.99978866415314871, 0.2730312020307436, 0.32438354425281724, 0.38428368411071023, 0.19185452412165371, 0.15180306026647494, 0.78184526782841035, 0.7985541175777443], [0.90409794298099722, 0.13699809504385096, 0.63109853791390735, 0.12912233175080534, 0.63468251678089427, 0.96993446269310568, 0.77400741737480372, 0.032570129085711219, 0.60280136888525326, 0.15225160164273088, 0.13245550151157781, 0.70312910973061471, 0.346727144385645, 0.18673523836311123, 0.3840669651265548, 0.0067039703320389554, 0.83350915902660594, 0.35799508913705269, 0.40642841966789767, 0.76969344779752913, 0.10415224344659424, 0.57121513424825365, 0.48343108373851529, 0.1182691014486621, 0.38924381369513672, 0.93613987917456554, 0.56661032316606341, 0.38584290598890225, 0.44737956343010121, 0.10865690859562338], [0.52986772527251447, 0.38477437153649419, 0.60236990351675479, 0.8837802336026408, 0.40771834386509187, 0.77293880610394639, 0.58509301346511444, 0.17153484117558337, 0.065213942800372848, 0.83659833398242245, 0.82199279329514052, 0.281188123386073, 0.61195060584323491, 0.25808727458120806, 0.2546024700162115, 0.38946876011415765, 0.14307684812055432, 0.3401447965826967, 0.12310816302935357, 0.17485947953683256, 0.066976583273594792, 0.71829680062218948, 0.40184893341975225, 0.86274513711658674, 0.36230856973368064, 0.8085103545480512, 0.73578295357617074, 0.34514280917288032, 0.59634203772006844, 0.10941371515019105], [0.79433200534156856, 0.61393616586413025, 0.4320363405288411, 0.0066954926562164951, 0.93264960494789129, 0.47798155528676822, 0.6530695022554257, 0.91186673169784271, 0.93024081195150721, 0.42214835303909715, 0.44402904207301808, 0.40839031010096605, 0.36299425196757718, 0.51030177624513484, 0.77723610309248636, 0.57525169404208176, 0.85707006757956739, 0.43443166342005535, 0.27818135730730575, 0.81136629784116343, 0.071720544106790407, 0.16857562207956611, 0.86830954456916754, 0.29378808146197333, 0.76300196406469645, 0.19888065164465019, 0.81689692559746441, 0.66850524435069314, 0.691470372625984, 0.97009274238347554]] [{"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}, {"DifYV4-nxtq0sN-wuB-oFeB":34844, "GNozug-82okoGf-8aO-TkWB":1398, "L4PABZ-AqRHpgA-odN-H3dE":99297, "LTH9ET-Kr3p37J-5bl-5heR":1360, "NqGfOw-U8ipFht-TcY-nHeM":6711, "TpqW7s-sUZVeoI-xan-H6ZU":98725, "iepuHX-L4U3H1W-TYk-6P3U":14122, "uBIEN2-fwzh17A-UTL-STzF":39166}] +18 [[0.8092757084352763, 0.08992259457487628, 0.18569280785172559, 0.75611467712240266, 0.36377485165404111, 0.64078001567212739, 0.097821453818684456, 0.75786548388710884, 0.46809393371445174, 0.77916698053229594, 0.71831147668606887, 0.2558786610182906, 0.65143035479830069, 0.88448044626726907, 0.15510983657522392, 0.10539377936596794, 0.024019638386299502, 0.0038249526301865622, 0.0799957636382671, 0.69475784262231544, 0.01944506732524065, 0.04491472639605254, 0.24990754910401825, 0.21855671163441104, 0.72700207742480361, 0.46425005383399554, 0.93492835330863366, 0.19731787223759789, 0.33721860803914361, 0.51688318236212616], [0.98159929440276161, 0.14457216156888775, 0.29882922836494563, 0.529595895297734, 0.2685485367831848, 0.75569447986567628, 0.016945004337063585, 0.46106895291888095, 0.99599504002867068, 0.37282672820396279, 0.30985679714106173, 0.22063209358255176, 0.59150599600025722, 0.76631836483711191, 0.3858556195820757, 0.83143987111478623, 0.011457893004749531, 0.46550349265024449, 0.80164771978771665, 0.5444276053375412, 0.87097039614495064, 0.97026581905499221, 0.72038843000784025, 0.4361587455828706, 0.14465348787672416, 0.17972564262956858, 0.80005389062375154, 0.85668307690773371, 0.14632832390827566, 0.28551597954130048], [0.21181460877739855, 0.33278309435782916, 0.11795215939927239, 0.19449767474391666, 0.61327077838184618, 0.17749335322238369, 0.036298740644152394, 0.26939611634089167, 0.38285409380014923, 0.10853577545972493, 0.18051532406707838, 0.8057251055582938, 0.059396504442058573, 0.63632147285585949, 0.59449794436875991, 0.19949608244103834, 0.724811104380503, 0.84642755987718432, 0.82509706629785651, 0.20259395412863002, 0.018026588524067244, 0.4719652265590133, 0.22048564921963343, 0.011333279607224855, 0.57476084371790925, 0.82801308373847093, 0.63259242053810016, 0.18859159015660154, 0.95041561562161259, 0.26329774771437919], [0.58332319867075266, 0.58104949776115178, 0.50231886642181378, 0.27356130725972772, 0.82636338235160112, 0.46044969631237642, 0.28273696247863755, 0.31656320456249043, 0.76467527525185974, 0.0914204677115662, 0.7255241319944038, 0.67042185020354583, 0.36837404290394826, 0.80829099106124469, 0.796971978951656, 0.38500901049860237, 0.078486309332762239, 0.46945996915162058, 0.325367037180184, 0.4903682345796, 0.67916612577221225, 0.850260691503135, 0.42395978490958874, 0.14663555523188554, 0.27426282306099214, 0.13389914746885923, 0.27849381321369004, 0.34669960548618783, 0.31370597031573455, 0.16492429550575405], [0.083139040762695848, 0.11584502956856813, 0.080992949266478931, 0.540822769522827, 0.57761840950102961, 0.8918818341645407, 0.175134456604066, 0.934712476375835, 0.46351269580965027, 0.9147703093581897, 0.29873451788379735, 0.26574330290439818, 0.95917310820552337, 0.0065095816153252217, 0.09845092358855434, 0.46369476620677808, 0.89036733326470141, 0.080065329803413743, 0.89490279324345989, 0.61039445197427111, 0.00772917753765523, 0.080370234299783627, 0.88616253826072222, 0.18296303721400931, 0.67227798949410744, 0.83093813173939846, 0.81228906851829641, 0.972524923436018, 0.65671884547250425, 0.43639134242899524], [0.12793016003335389, 0.00690612192021145, 0.57293001416971756, 0.47420775031285278, 0.4458416402606471, 0.46697409199404027, 0.35498970052919143, 0.1982524500119669, 0.38624198672408017, 0.1536567081616238, 0.30228888612178573, 0.3721664182394181, 0.99524013834841341, 0.53631065922678167, 0.418722942896547, 0.15244273109919637, 0.58014642125814975, 0.79299209564672524, 0.17164966335760812, 0.64902806780499167, 0.15029456291233767, 0.53931550683897311, 0.33824664469359378, 0.40554585038363844, 0.64907522940334017, 0.13496939124495211, 0.3070968255658727, 0.061744581437887636, 0.81620029413520157, 0.55558996408313288], [0.059275211681054762, 0.887532903802175, 0.82878812217989173, 0.82257730343147117, 0.065906314233182051, 0.71097094638080482, 0.835680314736603, 0.8945709661855763, 0.50810642127931716, 0.0018450137617092022, 0.12147042185428414, 0.74168489699882112, 0.55042031262580149, 0.51796060682730716, 0.91504594828690666, 0.047234137302644807, 0.88030725156080369, 0.97304675500795113, 0.7920146545870983, 0.61156920769112411, 0.36392681663225379, 0.64317216160120372, 0.21120713709718864, 0.06876599917092574, 0.46241968980766712, 0.69283160578140646, 0.97251742316997469, 0.96930263666522354, 0.0030746653796815959, 0.27236891681037168], [0.28666386830257584, 0.63048691267187873, 0.0062178023579853026, 0.11516063753153527, 0.7070230958259095, 0.41128142093742182, 0.53135538971560436, 0.44599209137612095, 0.38950967502825273, 0.22930548408806406, 0.9250473511889209, 0.5013157942957881, 0.018545031208548668, 0.20517656910675219, 0.42818338938359746, 0.857589460541127, 0.26547864989218195, 0.29298615503009517, 0.83177915700363425, 0.065649086773193366, 0.83039609164155626, 0.63558826794081524, 0.75810534249910366, 0.79708027250993174, 0.95906717340529457, 0.16951101328531681, 0.18790754642448082, 0.8228086126149492, 0.82385948510868612, 0.69668533537962851]] [{"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}, {"296skn-E7MANew-1LD-MnfI":88472, "3tBgrm-vmX8wkw-S3n-SK5r":14948, "AsvRBi-0eL23P4-DuX-7LPV":36661, "OcfeXg-P7zmeEB-eDU-OWXC":94062, "UkyvSG-Trfirlf-IzL-qrKJ":39574, "WhSgsE-JzSWhbg-DjS-4Z5R":31304, "XUWocF-LYNI1og-6ML-2Bs8":23540, "cqO4K1-9HKUVtb-JbT-hIvh":19849, "wcrAu8-0bPZ3WX-i5P-2D3l":81935, "xfAB9p-dqwYzIN-GDO-YmzQ":9608}] +19 [[0.19214938505481793, 0.1203246770444657, 0.22682061095121742, 0.60058123359852333, 0.41666223300525729, 0.10859332941909827, 0.46470160831520668, 0.86941153845249852, 0.45663333493050606, 0.51086711252431372, 0.50942562355559862, 0.89459441848345933, 0.41325362721123637, 0.381879330677928, 0.052761983830012515, 0.73072261693733631, 0.4855053961100595, 0.59134011258518315, 0.82286092628847052, 0.30720833022562843, 0.8581933488157647, 0.36446421974453491, 0.52815704589312573, 0.86886456411555091, 0.28842875010701974, 0.19960917399368416, 0.8007055942769391, 0.90426876784123877, 0.63241830605010407, 0.69808868356172171], [0.086369220452555684, 0.43144638484416, 0.62960485862541093, 0.23673376625685194, 0.45759446613749444, 0.98404444540253944, 0.49265244243139961, 0.21553183890854222, 0.0090858694843565235, 0.021047988515553895, 0.2394633698052645, 0.8206818125551516, 0.83750786483854645, 0.76922544470779941, 0.24736045418137353, 0.85766164818117929, 0.60341307398947241, 0.69987124546558854, 0.20976271683972492, 0.22941129526019721, 0.3269172336928805, 0.94004724789595806, 0.5321280983888782, 0.061493491542775169, 0.37821090313289618, 0.92340721226670663, 0.2166326329449112, 0.30910109524046736, 0.58981792729369176, 0.4968213500040467], [0.45399972171368297, 0.92003849986866271, 0.039581768829315411, 0.516400676778439, 0.4660897805866786, 0.27331206029650257, 0.33168060954940548, 0.69015403633605321, 0.62430470113660175, 0.66472242217370636, 0.13754245552574784, 0.05439322225641563, 0.5065577263960368, 0.068859242187711445, 0.34935211865133053, 0.0071342724872612839, 0.34950395554455826, 0.27115683311387806, 0.44591177466262821, 0.26412640097649254, 0.30292269064536481, 0.73417706500207258, 0.57393757150431979, 0.4517721503317077, 0.99958757145518307, 0.23904340209471409, 0.4074448135434573, 0.306085747288546, 0.21665605684802203, 0.42595823767098873], [0.063694515300421362, 0.60796902408130338, 0.82355511145147142, 0.84699284331426061, 0.22624481167306354, 0.052467347216412441, 0.78653510760639744, 0.039847927511421632, 0.15397045918143726, 0.22330071266903595, 0.7315095368723904, 0.43303963335160489, 0.43948434209974352, 0.72384794038768341, 0.45449309893086565, 0.60097443228065672, 0.2212153014755317, 0.30189480570725125, 0.81289632078054119, 0.50163376545131155, 0.050466584952173044, 0.49783980202325651, 0.63113007209946781, 0.85009727902995924, 0.76809963527651648, 0.94468363553794843, 0.610418660401025, 0.70789085808285912, 0.55505848246506351, 0.95301202897086468], [0.74541544822556072, 0.70115454603601735, 0.78677014237715226, 0.42531379616389164, 0.41916032779971568, 0.01988171556204299, 0.44352643602458475, 0.4613587995096492, 0.17311378225150942, 0.495990209436561, 0.60727369162444345, 0.239993823731537, 0.13044722445727974, 0.60974279987447078, 0.0098062674723040644, 0.89259652148264257, 0.18846508661411565, 0.23204287126479872, 0.68818088232168673, 0.11956098640980573, 0.38997903694611491, 0.2743886444328909, 0.55023842557362879, 0.24357077376035441, 0.33365046800050846, 0.18877727400608879, 0.056440745602801434, 0.24204862697234497, 0.089762007212484174, 0.43660772106254508], [0.28379985441569733, 0.056054337194134884, 0.8505750015302489, 0.628639667798844, 0.35349754556937474, 0.13403529649995305, 0.46337443073785456, 0.027322649385253639, 0.57586357169592972, 0.73621755533863709, 0.069498563430633431, 0.49831839815018353, 0.69470102803750267, 0.99905668292655414, 0.30409572471836244, 0.79979638655629492, 0.29751750032605639, 0.13905243286076296, 0.89904600264856593, 0.30609086224471704, 0.79904149826911575, 0.46136201242965513, 0.075430672085835426, 0.87656538338233569, 0.40247342994947921, 0.031031436242929034, 0.81212816967810464, 0.60103826877220079, 0.0962407253575791, 0.96329575602531581], [0.933559810169161, 0.57988111380647345, 0.053597703950848508, 0.29428259096255405, 0.97230465405583255, 0.7299091034832903, 0.76823004719167831, 0.30458620760985455, 0.93124745806072518, 0.84644095633806971, 0.88109476932603181, 0.985561713095734, 0.39511975625373941, 0.79741296636712833, 0.93204706854855168, 0.061822011677483135, 0.92273841854193506, 0.77719022929420534, 0.20437464109897452, 0.61330345148339249, 0.28894274658743713, 0.0057257944187022458, 0.42118194626006922, 0.87639779717415833, 0.93917470707151751, 0.12297585924153642, 0.2996357539297535, 0.24603214977122656, 0.96984398902346369, 0.7989876665031308]] [{"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}, {"5EFqyP-hYMfdS0-8Cf-DC00":34810}] +20 [[0.22723015667366742, 0.40942309462872317, 0.28492576086201182, 0.79017305587450148, 0.24232704437415198, 0.11447196385870251, 0.38825996621023184, 0.43351441214718966, 0.14355702268519221, 0.24975457610358065, 0.91170638842166751, 0.49546179118897504, 0.72433415000646428, 0.20938732646353675, 0.57192029994015869, 0.33054049354602955, 0.54942672554174132, 0.70097306844279772, 0.228089286079883, 0.76484567032571693, 0.33713079989145645, 0.67534215083811167, 0.57934412010664738, 0.13377981608827938, 0.46725714542337915, 0.54922883667486577, 0.53493463209850645, 0.23792403525811034, 0.31552358176190332, 0.79867240656094174], [0.56567235513719571, 0.37539872409241604, 0.43232796169442056, 0.82612756797862807, 0.49065026550726776, 0.38738979167825638, 0.80750897977085057, 0.98571408184846765, 0.7050072490009438, 0.088086570842380474, 0.59258559282853818, 0.788730912286281, 0.81251315400960666, 0.5410983949494893, 0.68809816897022025, 0.60550527270206655, 0.99100754138008662, 0.39148633826982548, 0.546057774134103, 0.6298426647337062, 0.1892337611007664, 0.8658678757532059, 0.38614201324693964, 0.47878910446951783, 0.43036713925353687, 0.11262172578203877, 0.77915959097907439, 0.56311166492769726, 0.080544259310863175, 0.2700408770227658], [0.2124380415750714, 0.28284968024021884, 0.929958183964707, 0.022352113824621989, 0.81917165645179668, 0.17748316227718919, 0.32959183823415505, 0.4318403033312832, 0.92740033180152437, 0.3917114213759163, 0.42842007722166642, 0.51988661777819667, 0.46417182647392563, 0.20552919336683129, 0.76391855344611925, 0.41054960648012928, 0.53657350312912067, 0.92123355501199411, 0.072975347749800967, 0.84738851111834446, 0.70370201301976876, 0.10018996648917156, 0.21080886811201527, 0.54754903278348432, 0.55227242451743075, 0.998630955746688, 0.49225404870510148, 0.16670107697586212, 0.19345098926936632, 0.064456836566695652], [0.60278589380428671, 0.61423917617072454, 0.209628534691346, 0.52595229914114483, 0.46590753031530607, 0.080226487401504287, 0.39723448657927396, 0.42852873892011867, 0.099038145434151681, 0.41388281242469438, 0.37532406060283907, 0.9570472632389202, 0.5417931213948276, 0.42798867261474527, 0.75955405079307969, 0.51897261296010988, 0.39855746663744873, 0.29439654494216438, 0.13729570803871249, 0.85755344515518817, 0.40151073556404826, 0.44291150386925238, 0.52201562249949085, 0.41441991522684452, 0.67076970955208759, 0.27912345340136213, 0.31368864183320821, 0.267434810900643, 0.025073017775296913, 0.59634357258956716]] [{"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}, {"Crzlwb-dLFD6VD-BC3-DEux":10495, "HNXIyr-OTDzuog-wrJ-i8s1":30488}] +21 [[0.80210647334191454, 0.69884789048694707, 0.67325247016421774, 0.18647318626057185, 0.71527136491642729, 0.96996035465464525, 0.86410361218641141, 0.43629253812037783, 0.1343664366155537, 0.08933635028895226, 0.26242879032691524, 0.96754901252159808, 0.31618543964171286, 0.783798165587592, 0.95305384434963836, 0.547585035257943, 0.0026095191527407691, 0.81746370905381027, 0.7398086347834486, 0.33751581121472762, 0.38505421156561148, 0.68032046740996477, 0.1327005237291784, 0.14702136790917075, 0.062344554245973294, 0.76303329680015064, 0.14792829349608072, 0.9089687593524568, 0.84690744774636639, 0.29694686594724218], [0.35228559989381047, 0.45214271127601913, 0.02905277297607789, 0.10093975614127615, 0.16651087021904898, 0.36618207228249966, 0.37024804851804161, 0.095731636375122386, 0.43139832867640382, 0.12489050877052621, 0.6636737690090182, 0.6214406422490939, 0.6359925343588686, 0.66240999411188428, 0.17397970770203552, 0.82563366392706627, 0.0089470450303853477, 0.28131582270999966, 0.64804515823834785, 0.77680712519867923, 0.882135053883447, 0.11251236565843969, 0.87467883890946474, 0.50218111527120635, 0.34985802333919269, 0.34387904030560157, 0.96357713629972552, 0.84888570040277445, 0.26750619321735958, 0.087936169930814767], [0.61249201752696247, 0.9240625842157949, 0.060282444734582885, 0.068371083488376949, 0.63644300659874808, 0.88210154997210866, 0.64560883362902888, 0.27108438435765536, 0.94394007249444145, 0.99617563956378663, 0.012093180158374128, 0.32748764737109237, 0.962567291660455, 0.27643503602774766, 0.64344046710238345, 0.67610314488631107, 0.23991024883071888, 0.37866421330192512, 0.027452162854329387, 0.483087235910321, 0.57569220986911462, 0.18490285542045171, 0.94826388394583694, 0.89311876723853445, 0.57711762716961656, 0.2314760028140882, 0.12422993236488866, 0.42916478033746297, 0.97180418256071077, 0.11007440556067438], [0.024370861593353554, 0.26295556338831882, 0.36131688598630751, 0.073971034731617591, 0.37305838448978113, 0.63757170578068412, 0.35320141430705987, 0.34415547941004865, 0.93561932467315412, 0.93936605070340184, 0.17054594295381098, 0.029112944089999981, 0.66679819752251324, 0.788513851666853, 0.9054838313743675, 0.147169861689734, 0.19952622338308434, 0.045804454562677344, 0.55668258346114818, 0.97177382583197947, 0.95320193196886283, 0.81785852110738977, 0.87074663726199331, 0.80777031105712072, 0.8486963543790379, 0.38377496033001235, 0.79359524765461142, 0.35068427938169544, 0.7315542219093093, 0.4632817891856269], [0.030258673446681428, 0.98121029751838873, 0.12536918495094362, 0.39978604017980524, 0.81016934480398017, 0.70355952568166724, 0.8640703755844974, 0.6412874571083037, 0.459907030502545, 0.9490076445522917, 0.48486434427696312, 0.36794178166355906, 0.79649030360803885, 0.13493323976896088, 0.0028343488802465844, 0.23712880449053464, 0.67168885185289862, 0.69483108013421124, 0.83370758176008619, 0.995201848986611, 0.22146461548252239, 0.27623009004106913, 0.18305758065455835, 0.13966442987932892, 0.50453190198409914, 0.941141583743334, 0.17821786368835335, 0.90634110074184016, 0.25016011669774685, 0.49760814300010103], [0.38343997395809915, 0.60317941374138551, 0.37772490843271467, 0.54111895984729064, 0.10933737096998908, 0.71564008528756129, 0.79914678876079148, 0.39110284733212952, 0.10441410230867731, 0.26820313127314388, 0.770239033309342, 0.87040559200233159, 0.799485270350349, 0.64233783239719777, 0.71811919624962128, 0.79460321221790953, 0.26505388781829042, 0.42335474809314044, 0.0392217444407168, 0.11319826254879861, 0.43721671638816384, 0.87687814001620057, 0.83608199795620841, 0.87619194588588156, 0.96942925501635357, 0.34024797204380897, 0.94094422582659754, 0.58721926933364754, 0.27546832167643, 0.15498035007071309]] [{"5j9i4l-WIO63j1-HQK-05yG":46266, "9T4jcW-qmJo8OK-vDm-SEKJ":43268, "H1lgcj-Udlhete-kuU-TF4v":22715, "IS4MMo-1Vby7UV-WUL-lyGu":3583, "XBgqLf-4Nb3Jst-mC9-FiH4":61532, "bXNFss-IaMgfzI-Ih5-gN5Z":93603, "rnsdUO-GTe6ufj-W7b-WqlL":49324, "tEedw6-XWQbwOa-jPD-bfiC":82421, "ulYVtz-Hi6kCeV-DwM-3evt":13356, "wZRkAe-Xd70AJr-fR6-nR1d":94383}, {"5j9i4l-WIO63j1-HQK-05yG":46266, "9T4jcW-qmJo8OK-vDm-SEKJ":43268, "H1lgcj-Udlhete-kuU-TF4v":22715, "IS4MMo-1Vby7UV-WUL-lyGu":3583, "XBgqLf-4Nb3Jst-mC9-FiH4":61532, "bXNFss-IaMgfzI-Ih5-gN5Z":93603, "rnsdUO-GTe6ufj-W7b-WqlL":49324, "tEedw6-XWQbwOa-jPD-bfiC":82421, "ulYVtz-Hi6kCeV-DwM-3evt":13356, "wZRkAe-Xd70AJr-fR6-nR1d":94383}, {"5j9i4l-WIO63j1-HQK-05yG":46266, "9T4jcW-qmJo8OK-vDm-SEKJ":43268, "H1lgcj-Udlhete-kuU-TF4v":22715, "IS4MMo-1Vby7UV-WUL-lyGu":3583, "XBgqLf-4Nb3Jst-mC9-FiH4":61532, "bXNFss-IaMgfzI-Ih5-gN5Z":93603, "rnsdUO-GTe6ufj-W7b-WqlL":49324, "tEedw6-XWQbwOa-jPD-bfiC":82421, "ulYVtz-Hi6kCeV-DwM-3evt":13356, "wZRkAe-Xd70AJr-fR6-nR1d":94383}] +22 [[0.33418430452769021, 0.1571443475012273, 0.87139187590553191, 0.763090618220186, 0.47775193122212678, 0.28389999727885518, 0.60928065723065294, 0.18254287191968643, 0.57947007579527943, 0.72593937626820182, 0.14287710926962827, 0.87271028812930207, 0.77124944454399147, 0.24053102139362426, 0.29016219903811691, 0.80315001498314653, 0.79098402862753814, 0.80971295720380854, 0.20572852915186635, 0.26142880150871572, 0.91710606956334417, 0.66677239944332367, 0.88713410049022545, 0.72340865706655411, 0.21737001845386472, 0.74304149549426579, 0.82270978751443569, 0.69775761928133251, 0.10737951478765151, 0.57959476319269088], [0.60299469128615713, 0.63858798321101629, 0.46925383993293845, 0.011130142791301534, 0.40098111747672516, 0.2359452832133756, 0.741963769539048, 0.34826007223526012, 0.71560910914008236, 0.26594076058061633, 0.46224879928794427, 0.35963507130215977, 0.66845763741676445, 0.7106414366916356, 0.70866122571945223, 0.086767069803651009, 0.44533823593898458, 0.042400966544069996, 0.78116483107884471, 0.41211516551293337, 0.18250392727391163, 0.40813589474299505, 0.23484915367653525, 0.34494203782165722, 0.71144439958151928, 0.9461217520263413, 0.0787026455601817, 0.18964430709762137, 0.096746371583170454, 0.62120335863466469], [0.54902166255379914, 0.11770550707253591, 0.82825553160603194, 0.77388503768587935, 0.37191383025405189, 0.17711562600570296, 0.86178445457242581, 0.044585536409431636, 0.8404884133437629, 0.4420466996702872, 0.44186889329586521, 0.45641383558216553, 0.34970829502918888, 0.83076879642307688, 0.785582121873371, 0.21814165957513088, 0.25551116287344755, 0.10123665028096873, 0.25817551647769543, 0.24361883975839349, 0.054690513307020017, 0.67147184745670674, 0.62932066984889656, 0.47469873261954509, 0.9181597571184491, 0.2058113884000603, 0.36298676227366822, 0.84181002822668294, 0.5526054643978906, 0.35828844057795095], [0.6644942845382964, 0.09963267693329747, 0.21416857011840207, 0.67085632929860239, 0.1850441663748178, 0.83490522956258062, 0.45267569233542204, 0.38686234444523804, 0.86702943045273806, 0.65921175796600728, 0.82607449280006051, 0.85912326339470313, 0.40269374312854067, 0.97166625503522186, 0.39211087049993376, 0.3071496344579735, 0.17420957630080303, 0.70473339014304748, 0.25791814598363616, 0.90951108421255578, 0.345006267184896, 0.15099416950029243, 0.84807081801231909, 0.60503072764194732, 0.98018613750157912, 0.30948576052223875, 0.83778928031758748, 0.52044145842104861, 0.68812448892766553, 0.18731499488616021], [0.89036346894801377, 0.64233269826174355, 0.29585735353310816, 0.14556525395910935, 0.43649747862395472, 0.20766095809371798, 0.49761861617043157, 0.688086845805612, 0.738158837209994, 0.29979369418777635, 0.2488957937091566, 0.057581232518764125, 0.39064986100691368, 0.3060004041967106, 0.9176315026313453, 0.6254600071904004, 0.23400063457263676, 0.21750051781164526, 0.84970624668501438, 0.34027924876054061, 0.36841664743632352, 0.23462341083308402, 0.057594406280263022, 0.43412109504206897, 0.59839613024996152, 0.777330379424147, 0.19909502256141864, 0.17711379114807946, 0.16626588327816816, 0.7952944545670152], [0.8772950486445692, 0.717352400414326, 0.078266125566924583, 0.22192857975363289, 0.56343186409376067, 0.94680227451033627, 0.6749488157830178, 0.5231986595219259, 0.12107880091446066, 0.89333805690933732, 0.35253590775876587, 0.55475285390756834, 0.993085741744833, 0.35489100872771373, 0.50301693643176848, 0.35984377903828368, 0.6019644373613654, 0.36374784123056325, 0.80814873610211513, 0.11571603655743357, 0.4099159749574357, 0.5973647170078108, 0.99615193128062385, 0.16757613990199494, 0.38645584070381678, 0.036630120414640532, 0.049782687149953819, 0.97695796687433811, 0.33959966381245421, 0.013330761608059905], [0.64059000868999438, 0.85524148441456582, 0.68083054114091157, 0.28564515016570746, 0.2565526283409767, 0.553278196197941, 0.71457679149234132, 0.93380928329130075, 0.088921630512443173, 0.13452218574330255, 0.46968665995218728, 0.67766994734202668, 0.24897810950254651, 0.071760439297789858, 0.42339434679406673, 0.68366061353723806, 0.29654798946447913, 0.17648194318608135, 0.20752951936365815, 0.050806008073532416, 0.37949871054978712, 0.79512025025328481, 0.53223052163765971, 0.94368108383944493, 0.50707551952765129, 0.68784915711747863, 0.11417878177368745, 0.77482101260376846, 0.60981797017613693, 0.14330709477937864]] [{"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}, {"863Ejb-khsjEOg-YxN-JFyr":92631, "jpml0i-91XVHKU-LHk-K8i7":12716, "mL5r8Z-srhv8Np-Lmc-AqEm":23457}] +23 [[0.21512619243621667, 0.81893079651557932, 0.98440744837687078, 0.53843802238875649, 0.18543355132489914, 0.0815671479746668, 0.35961220433253716, 0.61281807480648465, 0.66739946024083285, 0.78675272573864563, 0.24811238439506289, 0.30348528982686418, 0.30290214563319373, 0.6545970678746591, 0.37749638044505829, 0.72370899826241708, 0.52684526242281526, 0.6285903609383352, 0.89221266061764115, 0.19832330352680139, 0.9205528516890793, 0.11050075973282347, 0.29233630106450381, 0.5136567466714872, 0.21159225603577536, 0.92617995646459228, 0.24320486336590186, 0.035043182561499076, 0.37794153585512291, 0.36910969622694934], [0.56393807650058436, 0.54900032683837252, 0.99117535875164009, 0.22809122741083965, 0.957737645312871, 0.94985727097492045, 0.016351020940711836, 0.92130119550032952, 0.20799697651305005, 0.30955781610426292, 0.77553465053541126, 0.73100667700197619, 0.63163299129984107, 0.44456195564327716, 0.67741490543653526, 0.66155324322091236, 0.55468534334699, 0.8252124820689718, 0.617209535294706, 0.6278097586683552, 0.45511097084795749, 0.560630037742666, 0.65907786542921354, 0.52207397831518088, 0.20383465211537288, 0.23614406930703902, 0.72751032572407281, 0.20451353916514037, 0.52749941880984985, 0.58046363966043313], [0.55467356119742917, 0.61553716787209689, 0.41499301282485856, 0.2962362218766168, 0.057447943074404151, 0.018060877170638867, 0.69470970620953376, 0.90033902034305258, 0.65090157223942746, 0.84345498553657383, 0.42063954617665167, 0.089191520784699052, 0.74420329157041187, 0.73702735903991246, 0.022759871352225458, 0.96957455658930747, 0.81048443057917663, 0.17721900910588362, 0.40083666666810713, 0.4768852169661939, 0.97391728977913228, 0.47725846063268196, 0.27062639991216386, 0.44045319622633594, 0.015286004737699921, 0.068841250837361034, 0.10760154866466487, 0.049489056984012181, 0.0615830366960366, 0.71447199062731315], [0.97570579742021391, 0.923430305401204, 0.995672803405771, 0.69842578600352179, 0.37727818894743381, 0.59689630435941543, 0.7988213407208764, 0.10585700856841562, 0.015167384420374685, 0.99215871511446574, 0.62189896691674762, 0.015402919862829934, 0.019191184943811, 0.37097928668456981, 0.37032064970558376, 0.80270077224874337, 0.756478271452453, 0.33286177746966594, 0.98329652300631176, 0.780556555439777, 0.81886242607961846, 0.65821087690035285, 0.38972290081899785, 0.086074527693317915, 0.16038224960383107, 0.18258569287285997, 0.33105206983866631, 0.0658763968087045, 0.64407215273626939, 0.4321806949989786], [0.85573641036712622, 0.40760989751213961, 0.44961318870913713, 0.45086276057171559, 0.3103511722133121, 0.91926617926625487, 0.892045274777643, 0.10553783341525658, 0.079256957235261383, 0.16739664363650331, 0.20662480560062046, 0.16931624007356416, 0.15767159667905117, 0.15113889798520885, 0.76604478410979016, 0.65089097432243881, 0.88227854516911131, 0.81103331433612591, 0.013617655258359007, 0.16373854288719059, 0.78900676832689, 0.44993062141536622, 0.11965280306085946, 0.61019733701802092, 0.61937494540720028, 0.07786208052332444, 0.9981449017892966, 0.093173762502698487, 0.95480068674719587, 0.28356404728874307], [0.85702102108798117, 0.76217164399041637, 0.82253319940789371, 0.48241869754674449, 0.89757769973249713, 0.25053585227230268, 0.88737549765826529, 0.29389300751513181, 0.12737175450249361, 0.46724625327922642, 0.52012051845943019, 0.36112924765816745, 0.50571189908180458, 0.76929361556858067, 0.11663287573527259, 0.14115899786891006, 0.80203039079302918, 0.95961396609499172, 0.53076573756457324, 0.0662846515089599, 0.61479709078781131, 0.13075401904871176, 0.37981082492868867, 0.0054914379420462067, 0.9722712792522924, 0.10588450084895495, 0.28293639530981618, 0.13932416427543792, 0.8162691175344553, 0.11647474753207343], [0.80107664201920548, 0.20248247649386486, 0.45625744950743474, 0.39988601589885031, 0.96099320060778137, 0.54545937376093312, 0.77346765514801885, 0.60033745916566139, 0.76673780616045017, 0.61986771239639415, 0.55335601849418437, 0.21314288822808614, 0.028502443932686305, 0.30166962550118925, 0.82961966905888, 0.82481289035270189, 0.17185151780478103, 0.93234435411101535, 0.53411033057630886, 0.57056106330911138, 0.42958816609157957, 0.080100354967972853, 0.064370537438306452, 0.055949291470661411, 0.96401470191751415, 0.80155748009007877, 0.60415010275469871, 0.97085207160962439, 0.019105096636639951, 0.61889555737133162], [0.16214344426496752, 0.76161487346744849, 0.16011517652313079, 0.51828633282114056, 0.33032067182171754, 0.021227198355766363, 0.022413631068854856, 0.022495390867645737, 0.70560896060332179, 0.70752634985903884, 0.76182570463126653, 0.79386145046851453, 0.76731287617542676, 0.90589279025225022, 0.57553803030172479, 0.28343070246744972, 0.75754546950262192, 0.61559402324932011, 0.24562187645519351, 0.6578439355304434, 0.77307463189642611, 0.95787820812530944, 0.64648935740447222, 0.60764492855778685, 0.84437213673185729, 0.27911526992739066, 0.55310513018866092, 0.66899822326012848, 0.25128546013043707, 0.8537274017378359]] [{"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}, {"6rp1yX-GgFhFFd-Xvi-ns5N":73851, "9syzHS-b2jimvF-YAl-6DVJ":90207, "CGpIHj-jSjeSd0-PjW-b9L4":36132, "ICfQI4-VYvK6Ug-FEC-x97d":94732, "WHmZ01-4SsITP3-o1I-jsr5":82631, "XxmmrX-TWdAgPS-yW9-ONzp":99474, "c2JoLM-ccQBxYE-Iua-Vwa0":73149, "dNPicW-53rz05G-Wmx-USRb":29051, "lX7rBb-E0UOckz-Ww8-m20N":25375, "uZANlp-rO5b931-nMq-6hPI":68231}] +24 [[0.86425385615782824, 0.066358100833033951, 0.39543090643651169, 0.59218536318562942, 0.48110900493704556, 0.54672210075400984, 0.39615645246905495, 0.82739896249686617, 0.75606385877500171, 0.79976628493560875, 0.43599710181914531, 0.21265034900894442, 0.52263113184775356, 0.19236548383475505, 0.41860217344219808, 0.18425019002161425, 0.53446873982655274, 0.32346216208001077, 0.42438541143312536, 0.51941549065370329, 0.74634061707991017, 0.72185536105555681, 0.75598395945116814, 0.42185191943026112, 0.82809146209746964, 0.52658437019900728, 0.59803328654774446, 0.75944018729458107, 0.036788336180093362, 0.68155347270099154], [0.12978956053927904, 0.14045894491964361, 0.78800163804862389, 0.33717762509488536, 0.34807638433438581, 0.0040335343737816221, 0.13926911736769232, 0.21921104005674263, 0.43322787272996865, 0.79600820727573418, 0.92710612911567991, 0.69855416391783409, 0.22214563130230158, 0.60319718030492675, 0.48349599517001229, 0.175716956628995, 0.24028221735218491, 0.32697757244747416, 0.69861471380805118, 0.17974912271869714, 0.788309382016546, 0.59144522918607545, 0.69361459536769532, 0.27383102205655296, 0.77942418840578775, 0.10186222504665676, 0.52061128583589322, 0.45835164901160952, 0.37969188031543144, 0.095882462397487211], [0.1165894554455984, 0.26718297843255134, 0.016519426058596243, 0.86304999621560063, 0.2037626416426549, 0.16399255562951531, 0.9892990849585559, 0.047322006673414241, 0.95099280696969435, 0.162819274768498, 0.741456529080382, 0.59287675531454576, 0.98919553463801546, 0.82077010074124357, 0.83458350650702517, 0.74522395249153528, 0.80822764643687817, 0.88057807344835726, 0.803030382349052, 0.47764347544315666, 0.65357336260473375, 0.87689494618229913, 0.36127887149775995, 0.077235879005965291, 0.78821620961411176, 0.96257447655947659, 0.88661108988258108, 0.39537570736729455, 0.38840406709448905, 0.059835558593079985], [0.70603605954783233, 0.44411687132383615, 0.40158471478889346, 0.196076538102132, 0.47912984168599948, 0.60310710911834431, 0.41060996308572917, 0.54524550039744923, 0.88739170314543048, 0.69082580008686933, 0.92037633186309153, 0.26305389840345983, 0.91591285271293377, 0.54946816071703031, 0.51139123418079546, 0.67795353983924567, 0.24460048598146211, 0.5945557099282579, 0.98932292233716312, 0.77967992711251888, 0.65389482889087513, 0.98836136111171924, 0.00041398297869532641, 0.23329803456024389, 0.89386231351155709, 0.71049507931569678, 0.57970724634647675, 0.33906863740754378, 0.16574295694864249, 0.81950383561746776], [0.75235434093279485, 0.45708700222488396, 0.22147603980299413, 0.62609737366778362, 0.61784677039262692, 0.42052793579496694, 0.099422791038828, 0.33161922464051874, 0.31282980300128205, 0.37246455036432158, 0.68345045496973167, 0.92464022794933787, 0.870230257841398, 0.27861000074469411, 0.59773739797214109, 0.77405947650069951, 0.27074883691425666, 0.091965652639822038, 0.4311041948983293, 0.40885546456602129, 0.91733844719361912, 0.17524040669527308, 0.9223707838791515, 0.43263822887191083, 0.354110416653039, 0.12303999585211489, 0.53218559794290143, 0.38541093258651915, 0.29818008215695491, 0.56820712194956946], [0.7769670162704112, 0.63545679653393272, 0.95628447781148584, 0.90532634100434928, 0.80704389462041082, 0.061026007412017691, 0.15757447186500184, 0.11929055741033201, 0.97377619097032475, 0.36450404017380189, 0.049804355839040526, 0.067958562329520689, 0.54938279223734587, 0.35100129733698004, 0.97821198683276855, 0.63531598094077735, 0.33839859084052459, 0.39881152881075821, 0.69470307655323593, 0.14518481504059022, 0.65112805654592532, 0.65094434396689316, 0.22952352743387705, 0.30540960095944747, 0.99715832732915066, 0.16494502471385619, 0.598423926622024, 0.73858082435081429, 0.9009398021835876, 0.37247237702704961], [0.015390422642627244, 0.84773946269111344, 0.44163737217753607, 0.7206137108368198, 0.43433730006352433, 0.36413872417076254, 0.47590133986306771, 0.29367321762403931, 0.052725758573360038, 0.42902251283704596, 0.01490172480611518, 0.54717828649678368, 0.87589462844317234, 0.47251253488709066, 0.89591538892436617, 0.025105949287561269, 0.56295872122013069, 0.47007889413003845, 0.713950630161635, 0.076839299167433817, 0.55345071281302682, 0.38070180548709431, 0.960339093433981, 0.14212216028158819, 0.90845296037970791, 0.70071504160433951, 0.45292022746903815, 0.13028686046493498, 0.39078592817489266, 0.10345430826843194]] [{"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}, {"6MZgdg-PG6jlOd-9ay-TCgA":85988, "EASbYf-AkPFQos-Uj8-7Vwa":37167, "EehLJO-ZpwXTC0-wgX-BVU4":34791, "GJWSDu-RZIbMy4-I5W-IQYb":72067, "eNbX9R-XK9q5Qa-y2S-uFUE":53028, "kVPQku-C0qs0zE-XyK-mr2u":23233, "tOquS0-gyZvDaI-8gT-JRFu":24599, "thcprZ-WgiUNRV-GFu-m7Ol":27285}] + +-- !sql -- +0 + +-- !sql_parquet_tvf -- +0 [[0.2434200686509983, 0.24234602011681528, 0.64895448286354074, 0.80656526546697949, 0.33553233191049614, 0.93104045037487582, 0.46082051709757321, 0.53545409981555159, 0.11940161295457363, 0.50894827151143052, 0.82244549745752793, 0.6191370566860076, 0.33453321135998482, 0.075267042138438045, 0.40832636186584292, 0.19376042004217831, 0.46281496989081572, 0.54388378653102276, 0.39025393742796888, 0.39257363676724322, 0.70940372601174673, 0.89336874762060425, 0.35773689855687063, 0.46342745184481304, 0.82289964786839931, 0.11884358445222254, 0.73458918170034215, 0.16164855600390504, 0.7934128521328675, 0.76734686527974572], [0.21514872307532762, 0.4605570741471402, 0.069666541106819135, 0.70492028738367862, 0.73675974268412914, 0.40412098883943259, 0.57006914809817433, 0.81149134311857707, 0.029564542413484163, 0.48971676861864066, 0.42772759377026504, 0.62872940681697165, 0.84031514009422226, 0.31369342031813896, 0.10566261593964665, 0.5964036161859344, 0.5138291011545647, 0.33841236473208214, 0.24602629915282337, 0.40437292666699054, 0.7338776102306358, 0.54666339963226629, 0.37591059364035473, 0.16018983777385842, 0.87470842830327, 0.43783260127427237, 0.38237550446382929, 0.52640124030903857, 0.59507008866905564, 0.28992512490609956], [0.4906137706047905, 0.71698089895602046, 0.018517972540717631, 0.862522112228984, 0.33170845904568691, 0.25893021810096062, 0.086946580214511737, 0.74797772976901011, 0.57022889929989906, 0.72293517290282172, 0.44870686665639714, 0.39163874832166956, 0.47694942011079, 0.20606884449936869, 0.54833131681780367, 0.39659581979124714, 0.77826407943183673, 0.20988062448836364, 0.4218034645712998, 0.10445170058384445, 0.091410387129209791, 0.69881468554599568, 0.0694726906470935, 0.60081909208144668, 0.26340318020218456, 0.54675215456939352, 0.17812927994610817, 0.23273644898546708, 0.9716197201602389, 0.28357232035945179], [0.39441854927881315, 0.44491698100044685, 0.015265418569674116, 0.0389806629892302, 0.313687162692601, 0.24272318875770671, 0.82948922071003794, 0.84745207936479738, 0.84151880294739445, 0.87198460449683468, 0.88770647279663939, 0.025990169073353608, 0.609630015367714, 0.36925451158526956, 0.82272594449853576, 0.16550440916072373, 0.68346184886158567, 0.4808263638007455, 0.29356743977889554, 0.38695873606218245, 0.88571628064918251, 0.074808837828481245, 0.852394676517316, 0.90504678207366152, 0.67825081468092452, 0.57778337412379366, 0.32477763545864691, 0.61500027585520278, 0.69415865833439883, 0.15604596555016248]] [{"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}] +1 [[0.441213983188219, 0.11829553460573472, 0.33551661881638117, 0.15769240977086962, 0.039896571828509142, 0.84339289915401416, 0.29772970512610364, 0.68175512882934242, 0.13552490402879558, 0.73721186924123583, 0.94628908401158074, 0.71997162119921032, 0.804720529599055, 0.1151729366872013, 0.22392404886137851, 0.68585234720029475, 0.59439613229450683, 0.44486208624901169, 0.34252653767716268, 0.8562974946251527, 0.201746488817938, 0.45932588358867166, 0.1517964539013732, 0.74892856020576515, 0.50058056193959433, 0.26948611062925332, 0.1620437436263531, 0.82698516140777634, 0.52889327911424766, 0.18066087004085785], [0.010362183096527811, 0.78999638908659553, 0.56527131322878821, 0.25118644305052973, 0.62521031091709156, 0.11956844128064659, 0.090671740839035064, 0.37039189414085794, 0.46650530859147266, 0.86701370327619787, 0.257308726078998, 0.7786868003157752, 0.996655771716326, 0.89317874309062573, 0.4678275506522982, 0.50427955961462256, 0.97252972900641865, 0.90071181183763371, 0.80564182248990168, 0.14905217075470756, 0.23917456987234886, 0.63514306375666063, 0.35125400292656417, 0.82037477409142678, 0.4531757782350474, 0.57353369037566448, 0.86957535401118757, 0.7761514363463532, 0.5668618982603183, 0.26808269266896068], [0.95970626710882678, 0.527145829825849, 0.96413320187391827, 0.26696331606503287, 0.52133380723923828, 0.723644164135481, 0.85451183333088321, 0.19579339800924433, 0.31084787004749748, 0.14276983054377468, 0.90408211548872564, 0.15052724676616536, 0.43547344603811033, 0.48827227916236604, 0.11030881668576298, 0.026188858674662363, 0.86516116929528719, 0.51637029608157869, 0.61229299640390633, 0.78574995645380874, 0.42967394974211215, 0.20923834812955489, 0.46884389485729805, 0.95958809593433414, 0.31225284206616133, 0.33198129009681343, 0.43381692834016017, 0.85834392136876247, 0.75144045036335017, 0.72945158090582629], [0.41148698318265997, 0.080628601445879244, 0.51908547785444936, 0.14602044641194123, 0.58429759017063732, 0.90191591696415441, 0.75524762491909725, 0.18442255638470018, 0.17114615516780385, 0.042086027519590341, 0.90966685818779669, 0.25905105300492437, 0.3964358877453904, 0.12647055730932322, 0.86465695137615584, 0.044588323690071596, 0.52844566281610728, 0.87888197979543636, 0.79572794446681483, 0.97261514452511433, 0.7801388914181987, 0.714992559374499, 0.96393816592887827, 0.78684880328993734, 0.45635336894181222, 0.94056108369434654, 0.42792083698815186, 0.031918723473152943, 0.97838382176316985, 0.20138883091457016], [0.61372663360504942, 0.62109593784511574, 0.32028937889960285, 0.18051830601541996, 0.78706809118228849, 0.30515011469959086, 0.95082890257741848, 0.79307166320945077, 0.73524739424480867, 0.94751668871728045, 0.59074667941030223, 0.29025835124102317, 0.38499449770058913, 0.00021293843938408052, 0.96720187581129224, 0.75146429874269172, 0.65763746918694732, 0.47045135031494989, 0.27545538564950856, 0.41696777864529277, 0.5944726751300099, 0.96965493750497422, 0.24249763244399558, 0.19371005821830178, 0.67289749618156347, 0.14458810683840095, 0.52812170975295925, 0.45268144370116903, 0.277813589420575, 0.90457374897433984], [0.696016178629173, 0.92886645345961327, 0.49300409159889391, 0.37927540045969554, 0.14077260370492817, 0.13332451337850637, 0.73363346056273027, 0.4682811018658859, 0.16627224956727493, 0.19186531412796781, 0.25371400000272659, 0.17673319196098103, 0.87673810618486026, 0.43591107023251707, 0.23132814824794479, 0.67050643427900936, 0.663807544560106, 0.38852024304634591, 0.959591497213437, 0.12994434759112683, 0.85130162363223127, 0.01218078584309612, 0.65278621627934919, 0.032921463646273241, 0.49393740979443912, 0.21486286764997298, 0.51479014418195534, 0.45600752373650089, 0.41705027196238231, 0.64695489457111821], [0.010872207050034333, 0.34676659181731817, 0.33146247292351982, 0.764088673357802, 0.73412006722706558, 0.49073268630840106, 0.260956053085703, 0.97949249370683855, 0.86421094587619129, 0.073360849983796328, 0.57937707834508434, 0.48439425146535342, 0.97893239094856344, 0.32026056938771863, 0.67021925505543234, 0.97612870645029548, 0.8978046076683911, 0.68827624281371669, 0.593605481730133, 0.10098668029759172, 0.31235157468558294, 0.47548726218341542, 0.17390522997606173, 0.89247584555755188, 0.18901337317876488, 0.51995793603416451, 0.60579719821566691, 0.86554667105663741, 0.31838727138203637, 0.86507535550525971], [0.017406020453996751, 0.71346870431074272, 0.92843114653706416, 0.88357146720574919, 0.42621258947549634, 0.74642642807099535, 0.97945680990576545, 0.37283257603708175, 0.17675784716142917, 0.30953543302073228, 0.61518766050226015, 0.5608583568725195, 0.25873006173059143, 0.42286377292471278, 0.25684945029677164, 0.79523948117202725, 0.86984586123985153, 0.79717510536860359, 0.99620794195376239, 0.92678112846734606, 0.21000613127643475, 0.19649359393185328, 0.26866407178466734, 0.04490732896058236, 0.771527256070553, 0.84222333593723842, 0.26896056090646392, 0.27373918121173457, 0.32481954332433061, 0.18343776065277106], [0.20975091405949442, 0.79574725634859478, 0.38970417939142965, 0.89408704578862963, 0.3172260304544634, 0.13413682986387454, 0.229630752888198, 0.40025512371111405, 0.40818926260404942, 0.9956322900379958, 0.30937762224383691, 0.58922972998760126, 0.99941638604690142, 0.65208444146382671, 0.716387856656251, 0.60703250864004221, 0.16283173582476596, 0.9223331219817742, 0.26171337555152741, 0.81218859959906331, 0.75897725934669646, 0.30501427350229693, 0.84861066455870615, 0.0208980781474305, 0.25644698283290523, 0.70850930050577476, 0.22515164059733495, 0.15240451247521969, 0.076403780549422429, 0.52599207032176831]] [{"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}] +2 [[0.76900112854951785, 0.040707996366250665, 0.73040821209474038, 0.251174093959617, 0.71938437935301336, 0.90000184400502226, 0.29695811719826615, 0.77132447239782309, 0.3364963599585733, 0.22297486715776116, 0.69454649562099435, 0.9551249988985, 0.96884572098791266, 0.17347093634523736, 0.56434207544207449, 0.19757550008890767, 0.44749957994579836, 0.43964797103244158, 0.95970263505113274, 0.47711000474054077, 0.91446112100589083, 0.54755561537061559, 0.21819692563011628, 0.98157558927278, 0.24414192846210792, 0.77242356958590552, 0.73240079339151265, 0.96731327956198487, 0.68125057384504861, 0.79954643820947857], [0.39562102911013253, 0.300589975109828, 0.94439596733490971, 0.5141902766602281, 0.89030184402086388, 0.74879475016033337, 0.45192941244060825, 0.55873485232275011, 0.65972125084670563, 0.23822721388911416, 0.5150274008892608, 0.58530492398326617, 0.35768484262264877, 0.97596885014058132, 0.31274217063022836, 0.45343252631214759, 0.80627742747477127, 0.54524341639399443, 0.21472498857621281, 0.97483191843791228, 0.0083334940950107184, 0.82683567249404533, 0.54955269146584784, 0.19115719296286693, 0.55913969284327059, 0.87355200182544934, 0.2300649781304791, 0.58565227093746508, 0.88098357508802561, 0.67037094227057459], [0.74184683416858221, 0.075891462031071222, 0.064515003540916216, 0.049845173679730626, 0.87696615308627079, 0.11234846570742962, 0.74248755966533075, 0.90048070712959, 0.2523041212873216, 0.71632024444791964, 0.34429919572017365, 0.20523359092311955, 0.090291964734466879, 0.830771387202834, 0.28385302713803429, 0.37792742214106712, 0.95959426901562117, 0.92895384894747579, 0.1186104612260156, 0.827310353904011, 0.174255834899384, 0.42819576403401982, 0.48431052587327328, 0.67163330427016377, 0.33205533710830282, 0.090042281031522009, 0.65768095432516138, 0.051045457970155139, 0.036285958057279855, 0.25297016086547586], [0.9285677775991299, 0.54076758076254783, 0.18990926998900715, 0.44018741039784814, 0.57326240959177788, 0.754917124803915, 0.37700575623008181, 0.61256779139616069, 0.11240590345885104, 0.34902351431470313, 0.75367136432443849, 0.095559214747875765, 0.70970229598083967, 0.278686201696273, 0.55898642693518308, 0.99894462297013653, 0.021774384972731431, 0.35249162808059331, 0.826409859339634, 0.769622058295028, 0.544846499403834, 0.82921986740909126, 0.12692762586608886, 0.37011069574867117, 0.52054077643672869, 0.052535467884380349, 0.20751182007155211, 0.45927775678209204, 0.76092502158510456, 0.89077093170390753], [0.20192255322140196, 0.8668211791466981, 0.33205129332030447, 0.05507597451328694, 0.68561165139634339, 0.59933114145982258, 0.89557676946990661, 0.20611887032922405, 0.22246401434815188, 0.59329324492697988, 0.64965891159075118, 0.12769091521900944, 0.20833846518166077, 0.439184806779809, 0.23316981120375879, 0.14497903900676123, 0.75484713502109013, 0.3408550784891925, 0.66137484175617511, 0.39593523123493291, 0.076538452134858326, 0.64566900012301431, 0.34657328471287241, 0.52608608154876657, 0.34562557769543734, 0.38343336851759424, 0.52379984119104384, 0.87028921982413388, 0.26062881597456455, 0.38591313715133158]] [{"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}] +3 [[0.19201896280979724, 0.2277157594303113, 0.70696662663421062, 0.79613852089303372, 0.013154619183022187, 0.75111997454023149, 0.88382321644299544, 0.48943459157003355, 0.65226277171827529, 0.013413662629729872, 0.72326802798739775, 0.64237045403012538, 0.50375062994211028, 0.24899579433512853, 0.59781326778245125, 0.25528106793388783, 0.86173075992973081, 0.7486746409719387, 0.099299268038050448, 0.14850151396944167, 0.927790463055653, 0.0028725613850517639, 0.61728786534414593, 0.81819761160564253, 0.537622983127225, 0.61094125130920029, 0.60172562703852639, 0.87149398858891469, 0.71433858083551183, 0.29316580085759769], [0.49123087988432479, 0.34866866278473141, 0.25490758239544242, 0.198481216059325, 0.71511580179228973, 0.54975479458803378, 0.56106898112281323, 0.71362038603551881, 0.010263265208805161, 0.94870959378875108, 0.83717191918949041, 0.14835789294652491, 0.53323547998393184, 0.40038144179255375, 0.59480012365445789, 0.99670910532092361, 0.89349935355746291, 0.12382582530642205, 0.12766144391278778, 0.95538960556586761, 0.28541165853888295, 0.88009202376480189, 0.25010843602410593, 0.518535404496101, 0.35461442481978123, 0.353421537174577, 0.12878806551194355, 0.76162881474599053, 0.29874900496545542, 0.9918707222650498], [0.47930158736289308, 0.87777192682820793, 0.9729052229257763, 0.46668961209889614, 0.74838882464633516, 0.13589263595066881, 0.44697953455483241, 0.46935970588709519, 0.15703297491016721, 0.19316588128389867, 0.545124172609917, 0.78101350078908738, 0.6410163621688012, 0.11287592856305784, 0.386726865138738, 0.42102897746874313, 0.79283847003846908, 0.014142570868491022, 0.84386885813364443, 0.20991271398345579, 0.882542425059416, 0.011378082313593496, 0.13196624032401694, 0.26615522196368568, 0.91487698066175271, 0.98903826781794213, 0.45830797164952564, 0.79756295020852341, 0.46635555137147278, 0.80117489805872322], [0.76746156055211889, 0.04464106781443844, 0.7381555562161064, 0.68549383471345238, 0.14302482340606837, 0.38692219967019836, 0.85943425828802733, 0.038066621116226496, 0.3519608289695505, 0.43270586566689806, 0.980520942319257, 0.20754016648826257, 0.35075988224655175, 0.37849413859595671, 0.10657689719987318, 0.63663357651836272, 0.31193108009393855, 0.046479345312113285, 0.93969828894637586, 0.50390675553891506, 0.87483341624916122, 0.78675972974144137, 0.2210697010689755, 0.00023065065311922428, 0.87661405670073522, 0.86552932175105934, 0.19465852434769881, 0.51208470077970236, 0.77702964980771394, 0.05130887237216597], [0.850073837810945, 0.27775156946589707, 0.35518403819133615, 0.85835472931162415, 0.54459559773437716, 0.5326754255996724, 0.026723499549685004, 0.309702741762244, 0.45301312162045226, 0.76669468707848554, 0.30678889960312616, 0.14187528137346239, 0.048794797484269181, 0.4785560604824648, 0.52703128303907976, 0.089098475256317644, 0.48124329283070888, 0.73206112323895434, 0.9042557970716445, 0.25694889306889657, 0.28067359376324696, 0.893393269338407, 0.45376658932072966, 0.623667195113893, 0.297284510780336, 0.16061676229363742, 0.0422425514548026, 0.39194830607655395, 0.15927583233411913, 0.88403624434114714]] [{"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}] +4 [[0.59645537235209667, 0.60470950911235721, 0.90236107510871155, 0.66714780095213988, 0.95683682493491606, 0.0084652348887768181, 0.44903257208378411, 0.08657419553688217, 0.17840048948389087, 0.024768644008877794, 0.90679260937660666, 0.090295817453005744, 0.81267474389430649, 0.53225725844427452, 0.026670213937739828, 0.38148239227264857, 0.79269793518267984, 0.17868443924478761, 0.97028447888209113, 0.1496567720548948, 0.868275994476362, 0.27988042656300116, 0.47835050846162352, 0.07619909527486779, 0.71175422888807716, 0.638807768459914, 0.091001101168208254, 0.429455077477475, 0.81945551496522451, 0.81635799081909854], [0.25845611414342695, 0.93588080812337937, 0.54975845944769153, 0.54666909963684862, 0.599515977269857, 0.60291328537387323, 0.51085808764462026, 0.057126078723462981, 0.1405517516909176, 0.56476462877536227, 0.039572991518131762, 0.026839799812264697, 0.09910884735050518, 0.32409492347678981, 0.69713263993041563, 0.79728201439697854, 0.66393751595331463, 0.47052706818603907, 0.54869894274283593, 0.23488762718234768, 0.26720782241176821, 0.071224468160711929, 0.49044669171465083, 0.33781736901208692, 0.044862711179498138, 0.4492603646461405, 0.041347533684643167, 0.27770773470068966, 0.53750775889786506, 0.99226541237145072], [0.8263725295987735, 0.072117141469498436, 0.19693161240786194, 0.96684779209180416, 0.44364883344883, 0.49451042358532349, 0.93229631065503171, 0.0535381078552557, 0.53017612617495835, 0.14600645860097616, 0.75523242539775914, 0.46251916634464785, 0.61902877200653061, 0.83674681437105658, 0.56451851528597119, 0.042158274583507627, 0.815540183867558, 0.55462562167029039, 0.248720542423846, 0.75036780166853123, 0.41078590490136058, 0.33525396449642286, 0.57911156542625586, 0.22307758063375926, 0.2977524630546049, 0.289415268442888, 0.81538755260778861, 0.94461005454552083, 0.45889921113109666, 0.89290010087251326], [0.553140778444955, 0.70150761010622043, 0.85339078143198754, 0.39849248504695256, 0.10474699234992813, 0.584368818816282, 0.74809804480878206, 0.89101199561190281, 0.62426819902956632, 0.79150821255452342, 0.90853574748267352, 0.073699331422595771, 0.26216846589872445, 0.88376226554960446, 0.094225484142207327, 0.3478650714085475, 0.867084751299976, 0.94152179737902475, 0.98754263736534653, 0.12760503926495059, 0.13959090555373466, 0.2830789459171249, 0.025926563414585324, 0.61751781026145469, 0.68683762078068489, 0.017634678420189953, 0.60440356004121276, 0.47374298117945735, 0.12714672819981787, 0.726820587690857], [0.768692236624599, 0.86727171513467127, 0.25008380522776952, 0.99534657771489832, 0.82873131250276744, 0.47336375191149926, 0.25022765021774163, 0.34439378559597966, 0.28829068512851008, 0.82203788415633217, 0.22180125386949223, 0.36062986303979305, 0.10839585385643646, 0.2619857360401604, 0.42652382061133332, 0.45923190868591168, 0.6006171383174741, 0.51389169281847558, 0.32848864581573878, 0.77105944194000031, 0.18084506089320507, 0.41902416772213746, 0.027567250463190196, 0.10212846222502703, 0.804993201274559, 0.025096695934428759, 0.516785279349264, 0.77779481981211462, 0.15514080565785959, 0.59378001962267346], [0.13931096759109174, 0.45456305588268631, 0.54794734273235213, 0.46287887973702191, 0.72714235382191816, 0.65919531179500113, 0.0967414439562464, 0.95107024070960655, 0.16627495647351564, 0.89780128277916238, 0.87450744041810613, 0.65423286016217586, 0.552076437866004, 0.66684999799369915, 0.69706229920355778, 0.54978003184689039, 0.67379039427278609, 0.59354989215513521, 0.67403692534859727, 0.29904098633776333, 0.29853705384444806, 0.23480334473475006, 0.85984522310246847, 0.017618954535161491, 0.61287308656206507, 0.77194823983036887, 0.17764550411803548, 0.5697865206391971, 0.22391910519470948, 0.95275785041023753], [0.286368333266395, 0.13536189701183621, 0.32225775514574584, 0.15267755568543573, 0.21420587408132885, 0.39614265154377015, 0.96525156250005018, 0.35403516760008491, 0.39152755316908294, 0.19385604485780106, 0.952124029425394, 0.17496903468384561, 0.18200771594984544, 0.92271789224700884, 0.43109376824244017, 0.81907650123635234, 0.13282079376547473, 0.23707518579698394, 0.20404580115137794, 0.96229214910741256, 0.94308004342584906, 0.87823964362480833, 0.67567042589343407, 0.14526417408333436, 0.58426632020867664, 0.55450295032128571, 0.53169462264897926, 0.19750431518208511, 0.097213335953783675, 0.84445706310047719]] [{"LaLAYo-uI4vtyB-0Hp-hHdZ":33873}, {"LaLAYo-uI4vtyB-0Hp-hHdZ":33873}, {"LaLAYo-uI4vtyB-0Hp-hHdZ":33873}, {"LaLAYo-uI4vtyB-0Hp-hHdZ":33873}] +5 [[0.21999856488055747, 0.80106949963414387, 0.37888818235979371, 0.35621115078552767, 0.25905903110594364, 0.071486484085366175, 0.0873221357781595, 0.67747056809382589, 0.82675146444177572, 0.85778986301193283, 0.30961240678283997, 0.53293854613108982, 0.68386730354734, 0.7955796080221551, 0.78574708007062621, 0.28047617531729108, 0.020318084552748306, 0.311024599080163, 0.60445405220301629, 0.32359007828189013, 0.37264965333025513, 0.23379841929490885, 0.6143202190477125, 0.18205793005748117, 0.77837756124246793, 0.53844539297614635, 0.10183779363565648, 0.92238378788141, 0.87701308106567466, 0.01608737005588412], [0.827594077243545, 0.041497692464448455, 0.90690692210186918, 0.3004520621273975, 0.3292052136818755, 0.222308971558546, 0.19254566802257933, 0.95052386660978194, 0.12322597847003336, 0.86116851036162745, 0.76869393951040421, 0.22668843353210077, 0.068151646614525774, 0.9049898506726346, 0.24578262572798726, 0.78806300236832993, 0.68675973630725318, 0.32972487495192571, 0.885201873176727, 0.78577375258577131, 0.68842305374817314, 0.54208504069630747, 0.12366506456834347, 0.73026336262980807, 0.28065786702933149, 0.092844086938057124, 0.46256759838651451, 0.070280518678416, 0.1140538239204173, 0.35275389620574016], [0.72127280083399714, 0.6370248727762744, 0.86586544417322375, 0.22105031652880713, 0.14251426244787346, 0.97899028726172255, 0.53492708209058359, 0.51172091448127688, 0.73361228695521608, 0.14137020164834346, 0.49418706040976457, 0.39415230377827759, 0.52895982416633347, 0.62845853477625957, 0.22703849674065735, 0.15246962286528309, 0.16724085771160924, 0.12830084815246567, 0.45975791906683117, 0.57182621567457637, 0.29110243257535884, 0.075919701455123056, 0.094745057325779536, 0.46009708072614386, 0.79750880184650041, 0.28475920679372346, 0.947890468809938, 0.16987453936977426, 0.89376440953630976, 0.68556400758291], [0.79827990931672288, 0.49836024981190596, 0.24720026560360819, 0.67772774763967436, 0.66957780499577813, 0.01169281854023474, 0.25577548695998575, 0.72021704646774687, 0.57981448711366768, 0.66544559481048049, 0.61235444546351325, 0.56749680757393017, 0.21993494573022532, 0.57447480679533891, 0.88717707071425911, 0.99045338462680443, 0.15134942725446887, 0.92299290712958282, 0.81104803934436365, 0.91313197188281259, 0.14885949671895526, 0.29968797178357864, 0.30602835333328526, 0.64317558222495907, 0.25771519672292986, 0.19290497448364763, 0.0809204805995235, 0.2771724312750643, 0.98219271608465208, 0.98735000587042643], [0.78559599543873571, 0.42168973760716277, 0.77796388152736951, 0.566474052047993, 0.30122309543997206, 0.0865915690494371, 0.6816622920732831, 0.54363806097012024, 0.73270801172402322, 0.97930896151359914, 0.47843333112532027, 0.22035642249134402, 0.065173417541513623, 0.45705103685026793, 0.2818881950006954, 0.6013615049668, 0.24805379747880352, 0.078515713454836988, 0.78076499210593542, 0.1839264551324068, 0.67839597182795874, 0.037479387461347824, 0.6927327002951803, 0.78283004396700673, 0.66921098365878673, 0.75771478581566709, 0.1581564553425886, 0.35254123546476623, 0.50923058982698488, 0.58200694776128936]] [{"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}] +6 [[0.74898892346933432, 0.0772671311088804, 0.7484759208893712, 0.5151660040344227, 0.54793263204154052, 0.17860231571322482, 0.45191545547845136, 0.23965009227204648, 0.75616725349983827, 0.65085678938875624, 0.091046660666927348, 0.9603655949205443, 0.22065287928589716, 0.27452040044972426, 0.3514822877984074, 0.34117875302795775, 0.097458775756514959, 0.58830642952170442, 0.73846443555926278, 0.55919580552106707, 0.45037858572707812, 0.13647543794934003, 0.13255920726058945, 0.16871606874387091, 0.12323980217004848, 0.11077143650340115, 0.98716376144590612, 0.39502978088772089, 0.14875498433404555, 0.87662506786729366], [0.097177540615967284, 0.58062553328546673, 0.32883321513429076, 0.44066079247933532, 0.94338469633061894, 0.90187984272994726, 0.649590262283012, 0.39017853733540808, 0.84854017149809147, 0.27605888713519044, 0.87886616028454245, 0.21516276554977509, 0.5688472393701216, 0.586471076399407, 0.034879595323582646, 0.20950450881237637, 0.48845963793495772, 0.96786578730492157, 0.41171826872655148, 0.34050533532799165, 0.28483627140014423, 0.91468253233487573, 0.50068527422610676, 0.57436437151025421, 0.78922278893976228, 0.40151185527236943, 0.45609134864401957, 0.15023274081738092, 0.77922838014926521, 0.038137361409268711], [0.71990374589138273, 0.20655242892429937, 0.60941270448373053, 0.52078629013512845, 0.041731907170210247, 0.98077942068912782, 0.9109919617597787, 0.28672320734582168, 0.40895146831276097, 0.52722383207909718, 0.073219585257285558, 0.33108023790161634, 0.33633774792525339, 0.52762248565689507, 0.85669150222179846, 0.60282387017136729, 0.5554001710037908, 0.30519204080002893, 0.2827714878860903, 0.26101724567386131, 0.3906799976541605, 0.59529420909701591, 0.89352772091248334, 0.34304383954521367, 0.013593233816938666, 0.97447649635850464, 0.58301645554604042, 0.073216969289620626, 0.32397159130030639, 0.1045354893461603], [0.463444487297172, 0.12320660017285612, 0.42527426005426394, 0.10938220584363356, 0.64868713571057757, 0.62497350043169841, 0.53151368070810157, 0.46987168786856615, 0.41283387260934545, 0.63240813677967611, 0.67225902780435809, 0.79539228668720152, 0.91287911459943283, 0.029629459420203452, 0.74665101864135364, 0.71089222140789965, 0.55237067224930125, 0.5919070102404338, 0.31818226048488074, 0.88415471720921213, 0.1067743357336548, 0.019427107936906385, 0.0390677907616207, 0.910514798225775, 0.93150749994813453, 0.28463188844732745, 0.74635025896396823, 0.37084455767769875, 0.930040809814289, 0.560617829063112], [0.081455059011764974, 0.88351592241115606, 0.47216182668981466, 0.3310881195792611, 0.95073079898694557, 0.38218296970644983, 0.12143973746040382, 0.02161107241801552, 0.2665807749589495, 0.16860874262718806, 0.85612613249317748, 0.63744452192773826, 0.18711140333684173, 0.513394186725647, 0.44917145318950447, 0.44494417883418247, 0.50679418964395673, 0.707349534003552, 0.81073247699139583, 0.69158147875312181, 0.941457489312688, 0.50238175589847733, 0.39049146863822426, 0.723854279214668, 0.79799549183120433, 0.8231538671891494, 0.187302679262491, 0.63288189230629055, 0.92250126998609983, 0.47940516696581537], [0.52012767322904718, 0.73423333183112993, 0.88010188280814639, 0.36370414815398211, 0.47691547990367278, 0.77154578001562579, 0.035592991776508764, 0.28053841936003565, 0.8770300969408179, 0.72274788758822017, 0.57604170193939641, 0.12002936668819641, 0.6580216746373948, 0.48321040225048628, 0.54484563267470509, 0.56319130779392468, 0.828984464752151, 0.50190461026070765, 0.64130752055162077, 0.88553844661451409, 0.52043883831107252, 0.65465648859426762, 0.880604218971964, 0.71197519373258966, 0.20462071644749458, 0.25284276124900629, 0.20609954268305353, 0.76828857955440211, 0.86025657642428566, 0.10638234027949556], [0.79927111579241272, 0.77319412491072026, 0.71234786291599161, 0.98663510054265424, 0.142090640055615, 0.61976951058111007, 0.190183706978577, 0.66782683020346556, 0.85251866535275045, 0.90624277973045653, 0.915746300478831, 0.96487824207406669, 0.93584420334129326, 0.69348535876560369, 0.85986857055398813, 0.83102240017404283, 0.37292014826736763, 0.090192240263251344, 0.28494894692465034, 0.63035389280036469, 0.90091614453135149, 0.8966148738932973, 0.73072374474757407, 0.15845988891742246, 0.43303037476774986, 0.073845212252012016, 0.093542343140423978, 0.84854211055314921, 0.81067206111666534, 0.24803174961051633], [0.79978642793326571, 0.87094552308766038, 0.48494745484892188, 0.80754165858400517, 0.18051909146797973, 0.51076785353183352, 0.28385117255923664, 0.40596582722336927, 0.41714567162077, 0.2912322251847409, 0.63738659972639689, 0.84499601014539483, 0.84681275232183539, 0.79976200863446456, 0.504907913103317, 0.37858687746318287, 0.9914677636257816, 0.069205935699441334, 0.77729751106696932, 0.453447673036537, 0.40594647334095058, 0.66841840246964512, 0.11116279885676761, 0.78079189129873006, 0.32815105308790737, 0.36440839558660265, 0.99305629233320336, 0.42040545216651659, 0.1583268525896967, 0.33252772147316867]] [{"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}] +7 [[0.81016551924298819, 0.034827311984228504, 0.42584577044060956, 0.65761719473752112, 0.015828067858713157, 0.77104072310432981, 0.32378500814571542, 0.96337227876248355, 0.72757792965145829, 0.5567974933569082, 0.36826830177176606, 0.38974273844483154, 0.57162102356777855, 0.058465386858290835, 0.91864498625008917, 0.31216011334835558, 0.6550489555175163, 0.625780920785453, 0.43187750880489983, 0.83676673245429811, 0.56779250439812456, 0.9354600086277306, 0.62236094937422193, 0.15691930283260036, 0.40231063331435424, 0.899866096151119, 0.00014995422647023915, 0.36562233778094633, 0.82607965518188509, 0.28815416319737575], [0.15713066932678044, 0.54093839146879386, 0.63745513760888317, 0.53562895786030018, 0.64430740383211815, 0.15286762888333194, 0.03723667461329927, 0.90162644186557028, 0.66325757849073, 0.22788446447185562, 0.056273974448893038, 0.47704750973895849, 0.080940218893505778, 0.25126434204870973, 0.19370998073664791, 0.53974335953985364, 0.46942689258816095, 0.48752884465685531, 0.34096477617208421, 0.40927254387091117, 0.970756447679122, 0.25578312511509538, 0.49332202454146157, 0.26211956916091783, 0.79199773841736976, 0.7094144670119964, 0.58398713628800225, 0.33428542897272662, 0.0467374224120497, 0.83856614184501133], [0.014744462021776372, 0.56023108706724656, 0.47659267585312726, 0.73899662084823992, 0.10284855428016226, 0.97547443136160272, 0.30998152103917764, 0.6723784840374456, 0.71554214383956327, 0.14471256608577354, 0.14526599222519543, 0.080787918428295136, 0.82160180231952451, 0.46816069657117187, 0.28151705655704751, 0.51207966937590843, 0.55356922154497412, 0.46586980587080029, 0.36253155486899746, 0.36303896365060162, 0.644834999803446, 0.043105246912896855, 0.54836576700430339, 0.45364622361369922, 0.78249843461275792, 0.2994389768607667, 0.68353202464659557, 0.70848597457180174, 0.49993561085372085, 0.6600281602909065], [0.092137891217675461, 0.64027710938200755, 0.72226516782013284, 0.97962359091292317, 0.697630417397559, 0.9056963788579121, 0.97340955383028627, 0.79817118871038528, 0.13844041522017447, 0.93511256063770554, 0.55520696324585939, 0.11144059812794771, 0.43070509341155738, 0.52554407553194793, 0.71703685406104578, 0.8609350190402707, 0.51537934767855931, 0.33009818189400308, 0.948680516000054, 0.84440669507862276, 0.71942981008277407, 0.37734501702975853, 0.049644872382192839, 0.901622739944175, 0.75943770615630068, 0.11052520652694964, 0.55493656229300636, 0.0395965294284184, 0.42081983588158933, 0.94918502099866653]] [{"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}] +8 [[0.68849776030513066, 0.3945690554217266, 0.23162958058849181, 0.43005636234695066, 0.957092602038879, 0.91948228885547245, 0.083245187680261234, 0.80924594123290727, 0.59254527549686764, 0.31581885385248487, 0.84179038061634559, 0.48676570699450628, 0.18799174598170176, 0.270641957952527, 0.36534699824927042, 0.60822004432697108, 0.7963198854238247, 0.762516297939034, 0.9005458459567709, 0.19884478242750259, 0.31919896294241079, 0.012712557276409853, 0.73024089460035913, 0.40251978105790986, 0.16291771254423715, 0.48109078979891473, 0.8898869894955368, 0.81601245859986582, 0.68498718228715849, 0.25019864739496434], [0.47706590966593276, 0.56325831120710612, 0.8392537633568804, 0.43897273138057313, 0.85301917218994339, 0.56796091796708137, 0.34230675926282006, 0.86653241238575662, 0.34071052796556123, 0.0775203453187463, 0.63025410629159018, 0.046907238841265131, 0.86085475126953592, 0.86727564226352549, 0.85247880041860757, 0.73714060077688082, 0.31551359108809218, 0.7112736154030842, 0.15777720816833651, 0.68406360382435216, 0.42827382268542846, 0.64425237894734155, 0.5683208553966338, 0.91913527054612876, 0.18068631756127407, 0.25925536244278657, 0.80817031304045417, 0.93609610652048292, 0.94655916234729587, 0.78626718859228661], [0.5556108682202574, 0.35278401198088505, 0.91886192349020035, 0.97949015991126642, 0.72236746621866477, 0.20631298121534158, 0.44841199595301873, 0.904356155068425, 0.5277957617714889, 0.85991648821698019, 0.469178782728212, 0.56009547074301158, 0.33243540256762072, 0.6101093552752922, 0.27195424746462182, 0.89966076293354558, 0.42285082425776921, 0.47442298961060403, 0.50630370981911688, 0.21603083228172271, 0.19847308975831945, 0.312820761107925, 0.44012411210233082, 0.052395143897568164, 0.50119650232740143, 0.39000180450860733, 0.18680116291333215, 0.763447342317747, 0.14977191297792225, 0.72115313890638189], [0.82187086468070591, 0.60599165126723431, 0.0935781959392391, 0.060311139824199445, 0.054255165963644991, 0.7692933928946879, 0.9841421055158085, 0.18834590736272561, 0.39155517546397278, 0.069080912079281576, 0.3778159949220401, 0.236682237521175, 0.54454978465610093, 0.12383298972055001, 0.22931203174779091, 0.14322411701677629, 0.15296135845457082, 0.70796249545443135, 0.8133301087912812, 0.97854963648051674, 0.78271453069862207, 0.633733013645232, 0.7203064857308672, 0.80329598775419808, 0.5443275784228776, 0.95222272686000187, 0.11234847527342617, 0.921755099192636, 0.61274096850711846, 0.48337539696127818], [0.99671015744501623, 0.88205219529754963, 0.39080038943806816, 0.68615552604315289, 0.8767027982943435, 0.339881264507807, 0.051776876559952889, 0.662050906265826, 0.77372278007170359, 0.95062290273027383, 0.72071993668186973, 0.64864286537779936, 0.27911916560573691, 0.78377132982122077, 0.61010393198671953, 0.70469028857382654, 0.99091176016831517, 0.36394928187695186, 0.012141888831367686, 0.41415091265171733, 0.41790954896035248, 0.33439101018972717, 0.16830538242090443, 0.3437893755629261, 0.19256907979966753, 0.91162491708277071, 0.68600242534204281, 0.67749811976075147, 0.22397040406504853, 0.75418907030611149], [0.35303009836548194, 0.92018429752085418, 0.24357830973871497, 0.60356448441603816, 0.91233256198403123, 0.81505155230464521, 0.0489947551788793, 0.0721126097727508, 0.60275867288889284, 0.65033297719261718, 0.63469815771915339, 0.25065480826405195, 0.04592811661104812, 0.945007624693856, 0.31380375061728061, 0.38743343485339521, 0.36497865480445146, 0.097759315868661489, 0.88116872612816766, 0.75054547227932211, 0.25788133774464284, 0.9594137816310363, 0.84570429009634485, 0.92754160028737465, 0.11617439588611489, 0.44631605564678978, 0.76854688077028177, 0.23873454113186576, 0.17938077362471039, 0.25058683609112675]] [{"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}] +9 [[0.10701536380206866, 0.25884977813910282, 0.7902525709347652, 0.4137897115249527, 0.27238753268880089, 0.54107578240098964, 0.86478988051687966, 0.20059459807190805, 0.33888160879667983, 0.719477324458027, 0.040571298808728229, 0.46250918638629723, 0.31847082537786509, 0.67561422690960515, 0.29156123431457415, 0.8061135009203203, 0.81202731508154768, 0.29049496589672441, 0.44043930814534304, 0.091635859114874707, 0.30524389711018063, 0.26375964174118882, 0.96917096610218034, 0.79149270684693851, 0.36483301998482631, 0.17143524728940474, 0.62428000079801027, 0.22852918033053127, 0.23698658464922073, 0.54867063597387045], [0.59273164948907087, 0.49257470972557993, 0.15574411204597705, 0.87858494288438926, 0.47062560728205027, 0.19427997103940375, 0.0058312768690614858, 0.24434022139938882, 0.72443571207538227, 0.85680264464862665, 0.8526944763995663, 0.05877559267722654, 0.85332354997900661, 0.66876116653541973, 0.25187133028299558, 0.724416414559621, 0.0010912250011558822, 0.52597735117348732, 0.18050123266049278, 0.29639881386868017, 0.33762814721399304, 0.774880080189408, 0.89569369127605569, 0.22182645291225611, 0.132245952458921, 0.837044008027704, 0.017636902680343569, 0.011303049383206032, 0.075083694758589781, 0.9421887196075992], [0.6597422505383056, 0.26186025329317253, 0.067806183872612058, 0.52606513438745173, 0.483785364171029, 0.7952280018763106, 0.40110544187424768, 0.46000822943058528, 0.14491778836879343, 0.80136972804500617, 0.96405630892687511, 0.1705355346742442, 0.70958759228926971, 0.78443282657054048, 0.8891644462946402, 0.880223717583038, 0.27882128221618874, 0.796739922255595, 0.82698559109443115, 0.65519601923749549, 0.22998373144259965, 0.91617436860139956, 0.86548280291018009, 0.679905740410931, 0.20204620415799757, 0.17328860982752969, 0.96182833081355057, 0.19650181314388027, 0.19255753666336217, 0.16669198759951898], [0.6381452827072065, 0.84391472586830552, 0.90892664673960155, 0.41875005105217344, 0.607723568489499, 0.71090198625224044, 0.82228952564568591, 0.11835457718754339, 0.94453373516402672, 0.27764451333277906, 0.20914710604069409, 0.2734997654762199, 0.38640082394581587, 0.56208622657691443, 0.83412508195759483, 0.45362688091160452, 0.37099056632780025, 0.21006138606938252, 0.82662045491655223, 0.24017608978937577, 0.44379777663012632, 0.97235510385372226, 0.73476373876453049, 0.22351057072481229, 0.74861739990529053, 0.99719464826205706, 0.7020400619415893, 0.88535505414627325, 0.413516480057893, 0.63327013786090147], [0.20507586122469612, 0.98811685426650731, 0.77911527084437426, 0.98623523439110417, 0.81551271566249683, 0.15593468192732474, 0.58102414145868186, 0.30073987056329154, 0.0045531431570201564, 0.17702656134244665, 0.39457254069572123, 0.35039836562301596, 0.095678093621582527, 0.18288501538362523, 0.8775047397333805, 0.40969830878612623, 0.089198890272985443, 0.26709420017818186, 0.60510684697863015, 0.054706703738078621, 0.2317215475163229, 0.29502017333629404, 0.5822678777653929, 0.99353176512337649, 0.8842646024306704, 0.62535203070835, 0.83826735779682071, 0.36285011411972279, 0.45468602591212337, 0.21544849750619832], [0.8626208665227677, 0.44341062024919875, 0.90323893896095842, 0.11467045920962138, 0.024103412960169823, 0.54527831696765894, 0.60448963020282243, 0.66335735441477361, 0.53011065306112, 0.89268205668916989, 0.1544954975678744, 0.80983113681615193, 0.92980597932038311, 0.69868287353490011, 0.049055035017865345, 0.83378852277201154, 0.64204135889630765, 0.26148002644954371, 0.027005274786166877, 0.42166879055380291, 0.49782817293994419, 0.8370242575594864, 0.27195688879107427, 0.47806056192897739, 0.82717418241774965, 0.2467088676396616, 0.596520618477706, 0.27693823648719751, 0.19139089582272206, 0.664260928077592], [0.194993537448648, 0.55749451599402144, 0.46451552866889845, 0.79255663132007292, 0.54363642859110672, 0.95156273027575222, 0.601280115497062, 0.68893607183244232, 0.5520894328004784, 0.6930988947046306, 0.094099874296593944, 0.8266944815574353, 0.12513511119234233, 0.904919829061267, 0.81784179792683631, 0.59077864089307885, 0.880934188612808, 0.88387063583137215, 0.68867338745800089, 0.29731829697397616, 0.89002145177269132, 0.71282122545701665, 0.32208088048025008, 0.99947471877584815, 0.948764817309666, 0.21636778904019915, 0.66518483283860275, 0.79618896343245427, 0.081653726577266772, 0.746842556197577], [0.39977058600451942, 0.095765579912331478, 0.678697296545625, 0.53821054403318414, 0.76498972941588639, 0.30282722537408335, 0.91737719560873132, 0.39765972817570994, 0.33247802515196512, 0.6950287514280753, 0.8930134994084159, 0.31816678247820529, 0.18167416446232287, 0.83501987602299932, 0.18706636310467029, 0.19004762286451338, 0.86941278288640877, 0.72349795778809811, 0.43981033039793549, 0.16547820285294568, 0.56540450709376033, 0.04079290810557179, 0.86198206718683978, 0.38719781870870673, 0.73400435435902767, 0.247278890572886, 0.51941645273967563, 0.73108356279217857, 0.78270932799928439, 0.017008427649886526], [0.65534986643715021, 0.93201133404428893, 0.16911495471920357, 0.46431621455427396, 0.27698379935639617, 0.96339505081079757, 0.56059946318486076, 0.889741317592146, 0.72606585215382691, 0.823022754458958, 0.77479366490706747, 0.3194687597855066, 0.97765923070432714, 0.94128422023285463, 0.39393302755571324, 0.9415917757524811, 0.96693603366764069, 0.272611963219366, 0.14247854975198526, 0.652848839866098, 0.3307433306320462, 0.75693350464964571, 0.344930965992465, 0.32109039190975641, 0.56885585015474716, 0.24515962261076196, 0.039552613566138262, 0.041581863784819761, 0.77924208344792967, 0.17120982969349319], [0.61375007440586515, 0.61935152345941169, 0.80233577006151091, 0.22280288558541639, 0.1481599985894464, 0.74514266578917809, 0.8212179688689053, 0.014216482011297327, 0.0090191007768714027, 0.84913165673566382, 0.20777933041217667, 0.29644738870783327, 0.61218889497201412, 0.4144248324434171, 0.67934649370788547, 0.5536792063321212, 0.69724468730355726, 0.611448625240822, 0.63591983788508677, 0.92969184933094962, 0.89632694925215894, 0.47469905644263666, 0.88508473422551548, 0.30382339931803759, 0.859993084036579, 0.48491823030200787, 0.82982210628004616, 0.99332150425584431, 0.71109576667607277, 0.34261696770841676]] [{"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}] +10 [[0.046210800798201834, 0.19270931408029479, 0.031476592245683443, 0.8179910428536965, 0.80300002985709384, 0.374844428232911, 0.11473413378304687, 0.67074038967919913, 0.70269264562020894, 0.58458553043632022, 0.24137910691272157, 0.80798996757106778, 0.85186356535586361, 0.94555284981235721, 0.36073980750727719, 0.4308928939064961, 0.98812799047625011, 0.067403918230073523, 0.8970263709298717, 0.78300851419541506, 0.5158575887984499, 0.16702206238978623, 0.1543196468389032, 0.38326004925412438, 0.77608577478954555, 0.30347925784775487, 0.93006857232465612, 0.93402232634379767, 0.537570982307226, 0.99914277143516461], [0.20150651388014917, 0.55805630917060611, 0.86493219518590192, 0.758552422421868, 0.37146718694920788, 0.8353904249423203, 0.84496662564238545, 0.64220401216524459, 0.83840108676577429, 0.83488144517409224, 0.13370402738474862, 0.10413127331560568, 0.58996411431821649, 0.96507017885459079, 0.92444832210746064, 0.76018726840977058, 0.11036963231432773, 0.0050804057947642089, 0.7310018606002725, 0.61818358834502019, 0.13876616059119673, 0.11869224160162006, 0.33046322148151264, 0.57155348370503256, 0.7192229641257013, 0.003353573155513212, 0.31643477949556809, 0.63496334031807333, 0.542476061946612, 0.580776601320225], [0.97617141099400362, 0.4833038966263149, 0.79540748321193089, 0.79689878971802652, 0.62414740660923684, 0.34475145347985314, 0.60630131982647839, 0.16637083522764351, 0.099713380508174354, 0.19409605332859137, 0.70223957450251817, 0.6703664122374059, 0.86796083284677206, 0.49429240759711845, 0.079572212662333719, 0.49639222982948739, 0.050607085083005043, 0.442196309286627, 0.32921494921522121, 0.89490857918793054, 0.75765847561603394, 0.7887078545243611, 0.75560000969203966, 0.1199545824672571, 0.29080551224525708, 0.8240499273346783, 0.58142373704700667, 0.019603956153670832, 0.14594823573695859, 0.045668654237449147], [0.33990693774281533, 0.34696112937570889, 0.2549462461736739, 0.35925021065682095, 0.022102851120887834, 0.27353919760063039, 0.57563223857396506, 0.40260621950555908, 0.66885875522661586, 0.904244915035197, 0.96432409258903462, 0.72768765116822709, 0.87535691393107962, 0.816828411875863, 0.24651521014623812, 0.12774498596939354, 0.11073164068544805, 0.77853772761431561, 0.53911820439807145, 0.35466062879596993, 0.84607869352748766, 0.828851994510113, 0.17158300463234977, 0.83573656152026565, 0.23124424268215604, 0.8595488929362235, 0.0034046750273968307, 0.769728783660708, 0.46410062875460811, 0.67199567708517127], [0.54198238617846173, 0.15867755801256112, 0.055586382307430959, 0.24643517842384333, 0.027486210858807314, 0.91509026139169947, 0.90960856414920721, 0.61917156652442551, 0.57033317164463893, 0.17329168155590091, 0.508975590374705, 0.33094312180906271, 0.50206991675733426, 0.3292641100386936, 0.52684892313456977, 0.88340207422186434, 0.33190439599205968, 0.13505771934106636, 0.86777273960568491, 0.020504849796612823, 0.088824292013345518, 0.38185921507766307, 0.48316047736858925, 0.93774801067911218, 0.37318524186720559, 0.32006473685349024, 0.303187916625283, 0.71456893243575, 0.14184714869290505, 0.985210849808247], [0.28036230596193623, 0.36386234856278232, 0.10738247704220172, 0.025375177204636667, 0.84960801013553766, 0.89744716702706406, 0.64457524424388968, 0.458666011573034, 0.075528707542807827, 0.717450505168633, 0.26637523812493846, 0.028678003449522316, 0.37957704832857153, 0.46095882986695513, 0.056667034878263522, 0.47129925439590359, 0.018182770643203727, 0.14891883346932688, 0.92865861362550228, 0.058012255245735944, 0.41746062034714981, 0.12015358022974976, 0.21539000467968494, 0.23883962967601102, 0.06093955164774556, 0.76749218018286058, 0.89280734908028558, 0.072342885050724171, 0.87382561601856223, 0.53539411530574021], [0.68604341816330061, 0.40059051258986023, 0.95063020070153537, 0.98708204357627138, 0.41599065793372536, 0.40376969969812004, 0.032566700626776912, 0.0040799765619552453, 0.205948167303738, 0.25212317170720766, 0.35693833499536032, 0.34218631610155836, 0.58706972164936111, 0.06215111608768864, 0.37281030446317376, 0.83529382231078475, 0.1165345980110557, 0.25952570301105116, 0.34557987316486805, 0.51210225633713446, 0.85380887020074392, 0.75292341526863349, 0.057628087813190509, 0.77065111246784956, 0.92805984550963072, 0.030988327388477233, 0.65305753336446981, 0.54173348906723, 0.47373170603891934, 0.40889734521187138], [0.17159035564538183, 0.064634855560365634, 0.41148572006193351, 0.17150621829720369, 0.47002136521479254, 0.49727261091734076, 0.90815369167824, 0.28681273419927711, 0.039191390110807744, 0.087696401220295983, 0.86204863523926911, 0.69648150687510446, 0.69026631701875818, 0.045040431346139775, 0.42892208401761955, 0.92943286826795646, 0.20617734648260511, 0.56540347373812549, 0.429052463719241, 0.93000526611910317, 0.83239857352356028, 0.69176331465323482, 0.45739469828608048, 0.5439596299136783, 0.91580841987333483, 0.816917033408578, 0.65641599907653148, 0.063600950037368209, 0.091201226039362981, 0.6745696900560948], [0.40518874980815778, 0.59755059269797783, 0.16208623383376741, 0.6060778706714689, 0.62850596070081954, 0.801210677997065, 0.2257311666541838, 0.73898705393449726, 0.0362308764983994, 0.68071953834963839, 0.65197786759805276, 0.46838425900869018, 0.36578684032881881, 0.76620516216876189, 0.24117813343423367, 0.2932207622638171, 0.61963964243459013, 0.93940137493822962, 0.25866767424461445, 0.627142160196391, 0.36481901161199692, 0.59619909297283968, 0.0715170756428598, 0.67699578087331547, 0.3929720501020344, 0.6264631089749213, 0.797868179877844, 0.86206882358560966, 0.42713618514011964, 0.90145268357145447]] [{"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}] +11 [[0.65327432894692838, 0.32894198092623417, 0.9619307687845664, 0.99539125926505623, 0.53345764975478194, 0.86505853674749433, 0.18498367713182573, 0.11584283599472811, 0.19985143801523308, 0.022159199001309582, 0.606595360826325, 0.64825041300866781, 0.61877978755153007, 0.2018765340783063, 0.69951173647581022, 0.61528678466458853, 0.55314671839668983, 0.0642892129530197, 0.8654015226776085, 0.4606225640448115, 0.11353434062603562, 0.642277842163975, 0.13033174785619572, 0.9286040206797721, 0.03736384810835025, 0.74278867889748146, 0.45046214004600893, 0.17757070873604874, 0.61790238835819478, 0.18789277792224235], [0.07171479611701026, 0.7368287581787617, 0.97540744926463485, 0.85184717090624229, 0.23580777421094024, 0.52374078447814354, 0.53945748233680357, 0.50993633215001266, 0.27992450944595393, 0.90008926061840022, 0.194525950970871, 0.23069140239883068, 0.838423356611276, 0.66581465330948608, 0.5301133477109865, 0.41515538785953643, 0.422375081564913, 0.9634261700731509, 0.032259665280996885, 0.61518320061890575, 0.12634771552944124, 0.29782664808806236, 0.855017676653127, 0.94733490135742648, 0.591742726133152, 0.26399030213079355, 0.018475283902462669, 0.25974184070831441, 0.64211732975082525, 0.11887537646636637], [0.49417264233369018, 0.20639450380965241, 0.19831216872095392, 0.034151037233477677, 0.65926805176013281, 0.036622556612710966, 0.17983563329686836, 0.72276149814330637, 0.02756401723115609, 0.21710658242958947, 0.10383895097174212, 0.14504663675343465, 0.9985128921574451, 0.96958220041149823, 0.0041490798461749634, 0.16908340159150115, 0.60457953162803391, 0.42946735118633017, 0.11020947268541958, 0.38510765190517016, 0.88284556828327831, 0.40896336811684508, 0.44941108977633015, 0.7553212498698556, 0.5484983933690798, 0.68275185841162878, 0.16617936017644275, 0.41808477613518424, 0.10585553578239282, 0.30522817960137538]] [{"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}] +12 [[0.81185833049958844, 0.30666067502430072, 0.65538831497946615, 0.203106185079865, 0.00078869097953226941, 0.0031811868122340048, 0.265963761865172, 0.35560473239960733, 0.86359735984662778, 0.947279996928223, 0.21334787657408905, 0.86788450171151021, 0.22469454256582055, 0.60451736003698042, 0.97510032079053133, 0.39843263106573423, 0.94130813218741072, 0.0459891766655921, 0.1620075117346309, 0.64544667605197537, 0.67346080605993008, 0.12485844879470631, 0.66225160763659563, 0.7444093017507516, 0.10802675728658417, 0.77509559466729094, 0.47148574969643342, 0.606246096671366, 0.837093877912571, 0.034264285884160861], [0.10484242091193363, 0.10698604401641931, 0.49027143778178461, 0.83066693080528431, 0.026940343446078874, 0.24394496948341471, 0.65124682820104507, 0.794889767327406, 0.10724509563993656, 0.85880456857967635, 0.55080974485250633, 0.35676466588939149, 0.71067665182969331, 0.29456071787847493, 0.52172591541682145, 0.17099091509039177, 0.66714720709875552, 0.40116670543867361, 0.7252560430818068, 0.47813694280740593, 0.02875095147654827, 0.63379809191616776, 0.71478090015890627, 0.35246866813325151, 0.2196167947100689, 0.59801811756733336, 0.98890605566899437, 0.914027238765278, 0.91097426619299116, 0.28772354699913649], [0.80458524923354779, 0.4896063708219377, 0.7127715375099607, 0.43460258991311562, 0.17021432808035486, 0.39575488657685065, 0.56322366121268741, 0.84193842637233507, 0.11931250420003925, 0.48665763181268684, 0.51895851007557448, 0.31094285621035433, 0.077321141711810815, 0.86072402311973328, 0.500966597408782, 0.7844066212502121, 0.37721509212767568, 0.15194017226364476, 0.43535003391940696, 0.63030436198049256, 0.75403477721160017, 0.64551681664229055, 0.14290519599641871, 0.56709348878537646, 0.95411193456001286, 0.052092567453138727, 0.27786258850103562, 0.59279832111234865, 0.43167589510370286, 0.15072748398310443]] [{"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}] +13 [[0.12030761117169952, 0.21550887718555545, 0.49410894070815692, 0.42143467974520266, 0.81946279547105316, 0.45846426055389233, 0.65712742631890353, 0.15314477366832269, 0.41528878608123254, 0.12911772418269263, 0.67926389722739877, 0.93346414159677027, 0.93520902500503, 0.15421306709632598, 0.4625297969625134, 0.59796307240740187, 0.64657805418685965, 0.49975378919957991, 0.95603621000037775, 0.59442924775666361, 0.014895576314810866, 0.24932905647433645, 0.8320104371213185, 0.004921698921256823, 0.41486368996123879, 0.00877637834400713, 0.84954657070891948, 0.37800239275831116, 0.96528894570896462, 0.35803698662685224], [0.98656421020514684, 0.83663040078754358, 0.0043397755645839808, 0.84937980578720562, 0.64408353518608, 0.90864826936267229, 0.27342668654434943, 0.48574018492679705, 0.35680704565933652, 0.66448618463199893, 0.8359810167149393, 0.70374055096234978, 0.4801311131192485, 0.040653982267929045, 0.12166583771045647, 0.63302920220490511, 0.26609373380369927, 0.67778757995213024, 0.25706004985142905, 0.80689531575898643, 0.5893971218669638, 0.20846022706427592, 0.020614688926453684, 0.5586870942440505, 0.33952699176916423, 0.69266579298379982, 0.342688813351968, 0.054381340736341155, 0.15612317027708744, 0.81782523711478494], [0.26072892103325451, 0.28372249791459037, 0.75213869808471556, 0.55536400811949049, 0.63516508245504011, 0.36066994077238823, 0.54295766365991416, 0.19735228089662404, 0.73335375923466883, 0.16846182185812264, 0.67952492734942482, 0.084396098112331841, 0.99690539046973914, 0.72542520776749964, 0.869673045981151, 0.87991236032954057, 0.9219930806124127, 0.64473169653841311, 0.56421571979509466, 0.94611596332177261, 0.0027455923858757947, 0.714618162816727, 0.18300846926266867, 0.23349301928575728, 0.11458677859290856, 0.93516310042444328, 0.061061120316768247, 0.10822876868555731, 0.5473903903258891, 0.3135285066352077], [0.70591929252047192, 0.71100484575296885, 0.13757775037374376, 0.3416483145749144, 0.77491429511675114, 0.76577532102563506, 0.5937931142032723, 0.35866452632223034, 0.90451454807372766, 0.23857652469052182, 0.16254969822901377, 0.93498183181651284, 0.15389362561784659, 0.88412084792207946, 0.23824681625892807, 0.0056330166758079558, 0.36096357638862353, 0.51187010700685165, 0.3862168499310944, 0.72771673786372459, 0.15150431211997206, 0.75347941331206414, 0.51796803989538986, 0.93998990080316447, 0.69065321316005146, 0.2199474932592913, 0.63454721389116353, 0.662267623517278, 0.52090143900991992, 0.19950529810320083], [0.6369545971518068, 0.28810314746866161, 0.99187675214663484, 0.46621934301890944, 0.19180007937524712, 0.99629134412332487, 0.85457505635017283, 0.34448232018796265, 0.53589851407395017, 0.43505915421231134, 0.56633576198828772, 0.90600787788807025, 0.40525963816842869, 0.91737679718914966, 0.732739147700151, 0.063642831930431187, 0.98193529205762775, 0.20664390310775638, 0.89873627109446652, 0.370254617750018, 0.86644537098877217, 0.46907513119506405, 0.59370529976612019, 0.53114716834641618, 0.60401092009124335, 0.12242991663183334, 0.13376040628288555, 0.994904466084115, 0.2195809930672119, 0.12030147454189011], [0.1895226233581746, 0.50025502327786375, 0.54964791263575818, 0.6770005514826, 0.5386947235853734, 0.12549726588242349, 0.89810274375598942, 0.56013947101381911, 0.74016483810807032, 0.21221998126801911, 0.68889132378005458, 0.85634269507084348, 0.68999657767096323, 0.51649804083558193, 0.33998852400924851, 0.55008001616985336, 0.14954616728387149, 0.078438466371422155, 0.54121630298259982, 0.34643920135627515, 0.37207971614882851, 0.59489746800676224, 0.48268921049075175, 0.91572511596516992, 0.89664700730924707, 0.53038437285810514, 0.35117835910424855, 0.14109222761097551, 0.42660037814593355, 0.807613476286692], [0.58844312116110731, 0.00998861219290037, 0.4650307277507405, 0.68186046768271635, 0.98858741375534365, 0.50600886093536179, 0.040525031267945155, 0.32222525607820651, 0.70075511756166153, 0.22982447998082944, 0.42403275264536289, 0.69809801594245557, 0.59898726514736889, 0.50618006944936922, 0.21126128710859504, 0.60057132701195648, 0.28282143040615515, 0.41851428547131242, 0.52490343539402429, 0.016235801150929596, 0.18022113369038162, 0.55358455324863309, 0.3771328537445594, 0.61474240524935209, 0.83868159683975252, 0.23554515711942758, 0.42518564712515672, 0.81170982018846149, 0.51086526070376681, 0.68224837358617241], [0.45148014733742969, 0.72500056071217545, 0.56787879714221456, 0.826108957217412, 0.90460847899533336, 0.35508477260189741, 0.10631113462399189, 0.50769188321529624, 0.748667488335067, 0.48915600929197278, 0.35440940554012168, 0.37225471488555029, 0.68577843281665418, 0.15385878057663571, 0.12827807020774495, 0.07779506732223207, 0.54027842243752866, 0.92293253118975671, 0.64074379541355664, 0.35772483577421488, 0.22294904599228582, 0.023772496506828666, 0.42353712489606921, 0.07211266152200646, 0.22978018867740468, 0.98085300200016134, 0.98769883358254307, 0.62850673993529271, 0.072150585920248589, 0.48415383332080109]] [{"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}] +14 [[0.56756937645255057, 0.25931061792936327, 0.42232614384641554, 0.85459627004177552, 0.45020684777715936, 0.29767402545492538, 0.080229346108791177, 0.022898418006150334, 0.080332362797224555, 0.32680510799539164, 0.3650687261202672, 0.73586028167479645, 0.8973098989709124, 0.2578144961028338, 0.48980318727312011, 0.58474995051373668, 0.463153911850928, 0.011714371147868552, 0.43605207683782576, 0.63128663585656686, 0.066994296286058352, 0.60624375792077911, 0.030652471942043236, 0.62005011073507343, 0.48332817779826132, 0.084140408547301315, 0.47814224854841747, 0.23352396165508849, 0.34990896299393481, 0.77532205953952349], [0.76082076703846624, 0.441317043535284, 0.53053807222898774, 0.32391510683087321, 0.35807783006852167, 0.69920902070243862, 0.2586876845487861, 0.63079654219839232, 0.807970470981805, 0.73901699086123318, 0.8197980996131321, 0.36329070919547579, 0.736566665581185, 0.95688238564364836, 0.023256404137849462, 0.96588930115593807, 0.11436357246112405, 0.95542837061721053, 0.73947771682625274, 0.16933972668348973, 0.66412733451568728, 0.624340005062185, 0.33317430456438091, 0.84017318148392273, 0.54079035636850725, 0.25427021734002542, 0.91081164163890538, 0.63872062191874324, 0.63158133749035772, 0.57057742391975075], [0.91841487227229, 0.36089141138668757, 0.0833518326748457, 0.24625083915245694, 0.77530312157709935, 0.027465482056069579, 0.95410265258246, 0.40748218057585917, 0.90999291952892269, 0.75437735872720779, 0.20700262142983805, 0.88069329425735221, 0.776426506111379, 0.18844579712505505, 0.61773040604236962, 0.44643123138128971, 0.17184430446704302, 0.825230491550162, 0.5497603304387092, 0.16339366811118872, 0.45203552846210016, 0.83917282130172033, 0.96725134406770408, 0.904949199582363, 0.045364257947330566, 0.15090510395479428, 0.50148752549809072, 0.61086852357139487, 0.0038831854931207221, 0.33258867054162888], [0.864358110524949, 0.39858412816678412, 0.19801049814889682, 0.60936788065731673, 0.4111529183394369, 0.601147840432371, 0.16065988275179388, 0.65350426902971726, 0.19498974156276028, 0.0031566451467059942, 0.51737780925894017, 0.087480377307895263, 0.42647762629326036, 0.03126954097302348, 0.40095578528154796, 0.61334072256343375, 0.8547604341585473, 0.42253887073246943, 0.24214990009059389, 0.46776306359667008, 0.21987309156965695, 0.94163191048017258, 0.85255375069229955, 0.46582988114719315, 0.5913829324872949, 0.047119474081575685, 0.77133486882279689, 0.24119681994933717, 0.10457659612479175, 0.017360270140969303], [0.4344590756876251, 0.41633091478389228, 0.33299295423267627, 0.9381109781083472, 0.843190872002207, 0.34371843798919854, 0.29489092275591211, 0.12684395049835939, 0.058072439984518609, 0.747418627488594, 0.881280516441675, 0.022855548675478565, 0.025247720503955517, 0.0640659485109788, 0.31443421015326867, 0.30760003462143948, 0.95391189884505145, 0.53340023665257275, 0.098334918318898179, 0.036285552917449726, 0.70683106764649006, 0.12063067732060517, 0.80630626197886757, 0.85961986703067839, 0.44029065833304959, 0.16641629062876828, 0.96610900959379764, 0.993695368886162, 0.03549523253226905, 0.25497148760779931]] [{"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}] +15 [[0.65069461186175015, 0.72665034408710583, 0.77197434126543019, 0.40282943352113187, 0.64552393492322058, 0.765940755594443, 0.22553712288078731, 0.27967614535782626, 0.95223357133615949, 0.54526938922304757, 0.77496235494751209, 0.96470331564816869, 0.23770553340681233, 0.667868020195893, 0.71343109818229344, 0.59255155272705151, 0.15507542962345877, 0.35088308415654912, 0.46178941443891897, 0.24185122040952867, 0.35371780010244869, 0.47385360009386546, 0.69938190468420414, 0.044096788043858748, 0.567208730056698, 0.43637481904316988, 0.97552660164627736, 0.20175929227241229, 0.82018956127708165, 0.88760072011744939], [0.77077791579441168, 0.69166791769806091, 0.878063908524915, 0.60605770050101193, 0.13825734047055815, 0.84863795738609138, 0.552717342036034, 0.68031316989944823, 0.12271404474879566, 0.59951680568463339, 0.37295750752780021, 0.66936220946748792, 0.12861061735106916, 0.84926592031265957, 0.43012996051128904, 0.60616635809298935, 0.2884119839110979, 0.40839108314888073, 0.91737268012673845, 0.54859853111271917, 0.66838827558220493, 0.894979202023229, 0.9952304120654264, 0.87911869720148217, 0.84162510492326315, 0.24479501938510084, 0.13685750812493047, 0.707824882304903, 0.95656796082104734, 0.8193363965746171], [0.032795839549656147, 0.54309470598454945, 0.93430959898698562, 0.61444210103741137, 0.46832218569743367, 0.85985292742969355, 0.83896276248739732, 0.7409915362231202, 0.32561723874588178, 0.31757063898256332, 0.78284291874184575, 0.17443087503474852, 0.0085163469346685172, 0.1584218688292548, 0.95181871869239532, 0.75130563063171019, 0.13935976208647116, 0.75059457401479368, 0.46607078064580942, 0.528675033372055, 0.33546790889519307, 0.33257198228806806, 0.82763174348737478, 0.91622829227898994, 0.23702535173805139, 0.76127159583044646, 0.64846421400657173, 0.44111934742373327, 0.72866450746561673, 0.47024925088637426], [0.20146492800226823, 0.61198055097914938, 0.47379836397733044, 0.90374550757870242, 0.0916674759852596, 0.11127234794499319, 0.46375887527171478, 0.40817235193607437, 0.0060475880006942706, 0.46407032177532104, 0.27305596809932686, 0.934708669401926, 0.06771860929361917, 0.40658715728149475, 0.10757942520312835, 0.93499270409623181, 0.79393734469717359, 0.40917150245570577, 0.53432424255632982, 0.51647056880371278, 0.037223245839366093, 0.19622183932949233, 0.98922726213584167, 0.26477968619976855, 0.5750264400002475, 0.64939818730661147, 0.77396135668720067, 0.13761698286626478, 0.20576370871230509, 0.18071017457798044], [0.27304077465611365, 0.48786136981005057, 0.30767343146591519, 0.51850505530816027, 0.54934909601146364, 0.5832905828379199, 0.82182387420486624, 0.86207462065582108, 0.30920936464618076, 0.95605288448953252, 0.53174183320361468, 0.52954544173854945, 0.46737373946818739, 0.537124416482234, 0.32951155568702251, 0.5489121049450294, 0.18781708655579721, 0.8093028993601924, 0.88182652764419578, 0.12351799493042914, 0.32797809046648918, 0.14685949868905013, 0.73935953413090649, 0.54098447673842376, 0.67209791068212466, 0.10078998976063447, 0.46894956656617448, 0.88853341969470123, 0.20356518583992211, 0.63567767006747222], [0.8304862892999354, 0.8878444219650119, 0.93668353874422894, 0.22749988666965393, 0.36900637255423385, 0.26903254443142943, 0.50503555162680547, 0.29128108068104674, 0.047398055586663435, 0.366028377388672, 0.78662529981017837, 0.623314997721532, 0.70624375812058249, 0.034586878167038204, 0.3352389395506904, 0.41474949589797394, 0.20471304044733141, 0.56934753787614578, 0.28297101761010224, 0.92164733904388418, 0.070050855619523622, 0.027812872185748683, 0.68062177982366889, 0.061206729362587975, 0.36308943686791417, 0.25680830737538873, 0.95067848905191088, 0.044646154984623432, 0.25334429322945795, 0.96093131326474956], [0.24185188764374865, 0.65432728830221787, 0.48963811996369944, 0.35269327974971165, 0.99464648732477734, 0.92601156612453539, 0.55082892619951285, 0.28050283221305439, 0.95420121942474534, 0.80671473734134469, 0.7661016516966197, 0.24333431617559642, 0.1996448325495358, 0.82444538108099752, 0.10541527237050397, 0.36987117079037768, 0.49846835769094688, 0.92818278408078059, 0.658080431658304, 0.26510851467697838, 0.43992451041632041, 0.43477033054295244, 0.041074894112197224, 0.60884495973798547, 0.35322181124764407, 0.4539705851632051, 0.10858769985945937, 0.52818883966955277, 0.9742893147133761, 0.25374904530718068], [0.37041504704993122, 0.060149643547271592, 0.18914514645810887, 0.77492316432407427, 0.46921414751716228, 0.4705801400326729, 0.95241010594607223, 0.90763407400888851, 0.91525386007431, 0.80218069957878435, 0.65626568443862732, 0.20166865721607652, 0.03218389393473331, 0.78697222672484979, 0.80816300548785924, 0.39847986743588282, 0.72847716584134892, 0.17758934757320333, 0.86484765334229752, 0.54482325215440741, 0.27658260781751787, 0.626941148242353, 0.4657728037267147, 0.47624856901173263, 0.755458447422461, 0.47723196422221514, 0.11073101645313965, 0.14325514669026695, 0.19203938955821453, 0.20702959732944526], [0.5345427302854624, 0.59374014345641823, 0.723322848948313, 0.82901458510530113, 0.645897558165222, 0.42243909696500903, 0.29314345360539429, 0.23123825034142098, 0.91795196163706194, 0.1513787651662527, 0.49633682766464482, 0.31116374591329, 0.54504511329222427, 0.31493215704602551, 0.18787698157969335, 0.76445883110795176, 0.51290203654499311, 0.98888619948725409, 0.38570447720047185, 0.5594546354827572, 0.96895146962807954, 0.12500271418802067, 0.81334516289958048, 0.45338036542178273, 0.92489338328647719, 0.33087382275227117, 0.15205186849876295, 0.38792828194869866, 0.37540892606166032, 0.66944315645043551], [0.70208514375847, 0.23939990228801911, 0.37491340476014368, 0.64338417071823273, 0.68888056486988436, 0.19262845396691919, 0.47851099891324989, 0.77659816685596861, 0.89223455103779326, 0.95806868377131016, 0.22381709446788556, 0.10841353461395231, 0.62527170977196622, 0.15789015477575596, 0.0070725046649237067, 0.47563952441333834, 0.34484131944837926, 0.46960546425926364, 0.7899816518242524, 0.6734468595133104, 0.33612179162441869, 0.53471039849915591, 0.56296396786019709, 0.45811874308755784, 0.9815958237608855, 0.21500777471488219, 0.3860163603557687, 0.883035830944893, 0.098191402302366892, 0.51073285588808714]] [{"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}] +16 [[0.35441253712880016, 0.74230695184923845, 0.403537363109042, 0.41356457856567996, 0.069869469799674011, 0.26499751959177964, 0.26997623778465574, 0.68996237650176429, 0.68746833941028807, 0.91859472331559722, 0.92932073879157817, 0.68144804533134318, 0.30166772591661484, 0.3655925371376133, 0.33436780641007413, 0.73591415210585975, 0.40855237634638375, 0.70179921551863145, 0.67963263037462351, 0.5740227195464046, 0.23769397114716284, 0.55289746325245326, 0.92133186037084069, 0.73162152743413067, 0.97887992335314644, 0.993949924387691, 0.44737681664678652, 0.7791006656499333, 0.55486346057371194, 0.61609979618904043], [0.021674784824366577, 0.572114466357809, 0.98289037130124668, 0.68287939187360713, 0.53426339745090856, 0.046883000975285416, 0.040609178349845787, 0.46795450374648995, 0.27727884842198269, 0.36332622786411506, 0.90607249099812626, 0.54679565671210384, 0.43648620556398654, 0.52057300653524108, 0.67689010389158577, 0.026681964901016486, 0.23907579020911773, 0.92192647908806868, 0.19019452500968326, 0.44260716555915047, 0.7306567140780752, 0.75015580592098918, 0.36293230203116589, 0.32107142826908264, 0.46785507234112034, 0.42902406132396775, 0.35109546980254958, 0.21940647069208719, 0.50765058340243441, 0.89325063769159574], [0.18685925413111537, 0.38519633781118423, 0.95056266892615315, 0.86664551821219782, 0.19419351258197026, 0.048518746172414806, 0.95202280201738165, 0.85694240600697591, 0.40636747835268616, 0.75914779050281145, 0.55474692068297082, 0.11589596095833299, 0.41286794855345188, 0.11152145611466002, 0.91464830270759268, 0.38449827411068505, 0.54860395293395525, 0.6621920665988672, 0.49380761629765035, 0.8457080892075497, 0.10487107197440837, 0.8376417359484315, 0.57624364332686062, 0.34087578424987164, 0.28940139812221966, 0.15475905601162743, 0.02570455449560527, 0.57108597968421271, 0.99192588243623447, 0.13193711251291951], [0.54942637305173792, 0.70092878272876979, 0.022027625142207574, 0.85566720105355765, 0.85778166575656656, 0.509749721989981, 0.37766630090871178, 0.28566639954243811, 0.016968187266130141, 0.99421738781391378, 0.27560543252781966, 0.030991682207522686, 0.98036343197659737, 0.11608717839416238, 0.12907568483311171, 0.5102566998560657, 0.753898380275559, 0.21032807802177689, 0.39335908683556942, 0.12098195146344415, 0.90177919871708945, 0.42131910347185719, 0.21658038894996634, 0.2862728963245682, 0.13111336371703564, 0.41861042907965884, 0.13084375630872791, 0.021434617177623627, 0.50747214069654789, 0.55878395257768632], [0.59795407919148358, 0.968795734291013, 0.44814897593009861, 0.60623651003530055, 0.0843140462690497, 0.68592840339197536, 0.0158489296505282, 0.10324848835077227, 0.48118958357748132, 0.3406150339804499, 0.19277164072760289, 0.48848929544269626, 0.10938796755218372, 0.86532697150255, 0.7535263511254725, 0.10315050678818716, 0.25149411410468647, 0.054643785397782252, 0.075044437730137581, 0.2951863499687396, 0.14016265812699924, 0.050956245258205968, 0.72263109069516218, 0.93468017206214837, 0.59756436789679257, 0.15696010383345582, 0.93352476110085225, 0.78714510509527358, 0.068937779432996837, 0.93786204933055251], [0.1828091522323706, 0.35660557381301861, 0.622297542770797, 0.13188243339764161, 0.86124944258005232, 0.0065956638862136874, 0.59670342166609769, 0.97455485889821247, 0.72060079344830619, 0.39474952508461258, 0.97309485744339785, 0.1604850884612492, 0.149163680380216, 0.51544781857276334, 0.21454802976504617, 0.81564741833246523, 0.52883758289346261, 0.93502334129373843, 0.27691333586149114, 0.533959587479671, 0.89465025333782389, 0.12531826908045518, 0.72614334079091214, 0.743791706052353, 0.8687176796993008, 0.15392398510112415, 0.4152225576392552, 0.41322892123165933, 0.37281483289142914, 0.373224257864461]] [{"3tcH9b-tLgl3YF-UYc-oWZF":96074, "33rPqs-QwIgcnd-UXp-KjKI":85207, "R7In63-s3eWDRp-9Qt-SMhD":37556, "B5LE28-ZmkoTSE-A5m-WMrJ":59361, "Ixb157-ZePShy7-H9q-G3s7":27515, "aIIm1A-iXCXCbU-8nO-kzBy":49574, "oIDF5o-lHZOz4k-A7S-G3fp":10202, "EqUdAF-LfpxS5h-tBM-w9cR":17228, "wLoGIQ-tkE6y7d-kZY-Z9Ao":20205}, {"3tcH9b-tLgl3YF-UYc-oWZF":96074, "33rPqs-QwIgcnd-UXp-KjKI":85207, "R7In63-s3eWDRp-9Qt-SMhD":37556, "B5LE28-ZmkoTSE-A5m-WMrJ":59361, "Ixb157-ZePShy7-H9q-G3s7":27515, "aIIm1A-iXCXCbU-8nO-kzBy":49574, "oIDF5o-lHZOz4k-A7S-G3fp":10202, "EqUdAF-LfpxS5h-tBM-w9cR":17228, "wLoGIQ-tkE6y7d-kZY-Z9Ao":20205}] +17 [[0.97116668791348648, 0.47225829864239022, 0.48518800874288348, 0.12778390066476486, 0.66738280266453376, 0.69756208231553318, 0.58691193681800757, 0.83726240370669713, 0.6342386538366106, 0.74162195510476436, 0.18754811726602427, 0.72903016794612419, 0.029662560754477374, 0.61507586494668487, 0.81733460175545913, 0.8932036744394557, 0.46655828523063747, 0.82072526286742431, 0.77657049415927248, 0.833949582656972, 0.12484821467354346, 0.88577116031139791, 0.982082275780593, 0.23944382406497233, 0.12939781292324037, 0.46834988139007749, 0.32268977838427482, 0.2363626321919553, 0.20666984843600844, 0.27802324450910476], [0.20302386922242821, 0.60147096817582912, 0.0050258369610359788, 0.61726051685264161, 0.030242892390883958, 0.79838032598306841, 0.49191411112674843, 0.82458213566827931, 0.62964026399704942, 0.50930647233469706, 0.52575255978398172, 0.27991823983197217, 0.96283089772297081, 0.53966525303108481, 0.041135728333564159, 0.91886804184433779, 0.06614990867474857, 0.58760231232605664, 0.28306543709178233, 0.39492823425804258, 0.82709132506633376, 0.391060440161891, 0.17922493429735875, 0.12739221563621561, 0.073061039681284634, 0.19174649205667915, 0.24888638004519059, 0.47750400396515058, 0.45941269445820221, 0.2883667791065001], [0.412862362617056, 0.20658440923480914, 0.812681024320184, 0.69075828744408785, 0.69098186968779607, 0.18722866616986078, 0.910121946725318, 0.53205101612402728, 0.051892087705240231, 0.062308663783570806, 0.1581147555397675, 0.94775392606693765, 0.61455697567015721, 0.17639511232209948, 0.26967631560557581, 0.58866504991466184, 0.99164457089824209, 0.64034790405812858, 0.83528295921027806, 0.79876474658801422, 0.81206485622870717, 0.83220579232098246, 0.63139096511723591, 0.75975770452747227, 0.36324044928581278, 0.98613934801669934, 0.61749175523892541, 0.58068827636043818, 0.853262783655353, 0.77611382058540312], [0.83523333303482183, 0.39406071393729658, 0.0089066454926266525, 0.96204153750429555, 0.84208478777363882, 0.0091541147655833655, 0.59754608594568348, 0.23358624192734911, 0.88721581050952136, 0.56872970501594677, 0.98933581843314311, 0.36041433656082, 0.76870907955883094, 0.95708550027754857, 0.060591026509963841, 0.63757116316855633, 0.19891764653914001, 0.41542171440630193, 0.58012062582148582, 0.40079318790378848, 0.548725962946865, 0.93473789000914764, 0.71733802008541325, 0.078949461788597963, 0.041667580665553205, 0.51699022719019438, 0.46429387061193617, 0.84943892188170933, 0.825298235177949, 0.3219960034289634]] [{"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}] +18 [[0.3363835250565157, 0.42751606296856792, 0.95909113566627757, 0.015305717549693898, 0.63963346692763268, 0.662206475764985, 0.19265081668227968, 0.36648829855161447, 0.67768234725844145, 0.44679057950003875, 0.94253566163541913, 0.0419972030351593, 0.88215408366970394, 0.63119869767184589, 0.19829737844392692, 0.94022059322342155, 0.82699301653092216, 0.67549611064588866, 0.20717354676738475, 0.73738381032610623, 0.739135700713662, 0.5276357210213487, 0.12251055206492134, 0.87199014141755016, 0.47051403840344674, 0.94930513747355894, 0.82367703710708573, 0.55189507748263777, 0.78847589969397591, 0.42454220893889616], [0.8138707834129465, 0.34691087630813566, 0.10440491569846055, 0.86880075736393825, 0.31804157739550953, 0.090970893430076027, 0.17926605644194693, 0.84540648442663369, 0.57522357752851816, 0.88526616315031581, 0.91999911493853148, 0.89185035004328017, 0.8478905507068244, 0.96739925034111662, 0.11212771356189832, 0.86012482193249007, 0.85551778689317015, 0.091483510347962649, 0.82164633304019841, 0.8833160392196544, 0.18035492017878907, 0.85024835198148263, 0.24319365806536042, 0.69492302800354078, 0.69048416232444165, 0.062594312969588173, 0.97171543356199541, 0.54788250558115881, 0.17855857679116593, 0.94867419350117033], [0.031265477524745844, 0.90703607725982471, 0.22418561306493434, 0.31945959704555194, 0.50728920198202476, 0.42178393253363022, 0.7537411843487577, 0.4188848361868045, 0.29767846296246736, 0.42492751242319071, 0.89347014952084336, 0.3740431241351827, 0.32308253514173324, 0.34921100551352935, 0.49004913935925043, 0.432542693822872, 0.51840547547479865, 0.0048294863960115508, 0.10735800737129741, 0.29601321937771352, 0.065114536328428341, 0.619179455681096, 0.39902660832583658, 0.894886076529092, 0.61793068205933577, 0.904384983760988, 0.50339400846029669, 0.89195431743719056, 0.72966676443254186, 0.18864113588467379], [0.510837178188414, 0.050851241666467484, 0.28442465098548908, 0.82417322635675327, 0.11299673914897179, 0.36046971621512769, 0.51205796709114448, 0.66761917185536568, 0.25924801537811126, 0.46747610082172786, 0.075751868639053366, 0.417899658739466, 0.017835147054534151, 0.5364567488079045, 0.79976731528571987, 0.99214819887466188, 0.86740719587880288, 0.13086695912257784, 0.83584211602622527, 0.922031913065875, 0.32926466584486569, 0.087835319690293567, 0.45443662576789678, 0.96080095120271691, 0.19144601980397902, 0.64600948804752578, 0.19095684882450137, 0.70566245630404956, 0.5728533274540859, 0.27801163400030726], [0.77728368563869421, 0.060582773303465065, 0.72893703993785919, 0.777705285630967, 0.14538181800014738, 0.791379878100802, 0.81002972551888275, 0.15735777533410567, 0.80245711330091174, 0.25797477626186638, 0.74923949108924448, 0.82091500916045468, 0.046077216440706725, 0.64792964887186555, 0.078883565072431439, 0.29698884916744528, 0.81286449013061379, 0.68090736489166359, 0.75276306577383434, 0.53878953112438455, 0.4358166258751397, 0.80544546452824861, 0.98615047603116879, 0.42811854847808894, 0.22300491979365511, 0.367445045193459, 0.71618134309389181, 0.6566704685854744, 0.56434590796903084, 0.09403301323320945], [0.53241208628156977, 0.60027471712462266, 0.22333716095732925, 0.50696411100162109, 0.83062640492685158, 0.15034108225284615, 0.57709347032964831, 0.556486677106241, 0.5025426668237456, 0.72165601342526153, 0.83071481069805364, 0.84960399281995824, 0.689895357695887, 0.81680887307635319, 0.65096302895655045, 0.33592101171909383, 0.0034336204504900047, 0.59491083971015712, 0.23941933417788464, 0.20977397944338083, 0.13570056950551224, 0.86590131222021416, 0.59673997749755925, 0.08909803039028541, 0.98102711896801353, 0.81162003517155912, 0.61732282767271018, 0.23949634123464703, 0.21699446757510632, 0.52095565225998386], [0.69764304363933849, 0.72623132400984125, 0.79868991456783378, 0.11876428171850673, 0.74171116896387967, 0.89736700685986859, 0.9704003406749856, 0.86688442682327971, 0.8035520524197548, 0.52804879590935438, 0.2089589358298638, 0.12045775741330678, 0.11929919630696029, 0.288915336441934, 0.97878029697171542, 0.43389741515493796, 0.95044314151513132, 0.94089461782601458, 0.30068952260281412, 0.42822900954821463, 0.48496903937595859, 0.80414392686868852, 0.91497175493206417, 0.25516183528921133, 0.024562985897052303, 0.60065225353092178, 0.25363790279137033, 0.62313063409363911, 0.82376272930663241, 0.72802989535747908], [0.527723699648675, 0.38668165406839161, 0.24087278831596093, 0.38626946164413589, 0.9825088472526472, 0.64295856221293435, 0.31066053161219964, 0.74976892719019261, 0.776863583047958, 0.12767745731550328, 0.45529368986375507, 0.5536664313027847, 0.0025000685784795085, 0.45786271193845229, 0.81267627065609949, 0.046543034531495775, 0.074726867753082793, 0.42974796956612737, 0.58028464764526932, 0.25080874451564394, 0.098374337218951724, 0.14762987275914652, 0.84780057536505182, 0.867593766729821, 0.840879742885282, 0.23309462941409798, 0.62328411343587919, 0.96472592398973878, 0.62909073916269764, 0.74392021613594894], [0.14532367994551953, 0.79438161784813077, 0.8271665570909329, 0.71809037693135824, 0.1175467019161498, 0.7012570888990286, 0.55002614548691819, 0.45073863497133926, 0.2020688508478502, 0.10261312595484451, 0.9201133091909105, 0.71047394745800707, 0.96490608550321555, 0.94830503617426753, 0.4848185191583253, 0.1064222826975858, 0.47906770866275383, 0.9019171922329241, 0.3777183617600689, 0.7284508952780494, 0.95999558004129593, 0.72337962158359159, 0.42886277922067195, 0.00042546285601285927, 0.13320855344729543, 0.97895110860912082, 0.7076614721641864, 0.4099825893228175, 0.5154572485891008, 0.25621219776718074]] [{"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}] +19 [[0.064110097878604777, 0.20880075639202911, 0.8271834124783779, 0.5409920484990518, 0.970148587111359, 0.8291199604200874, 0.044393498739233261, 0.91190206355989478, 0.19435805338209444, 0.6088772957950721, 0.12166195872864705, 0.7365113753875705, 0.60576251085398969, 0.62022100824175985, 0.74187300322342165, 0.94936690438807747, 0.78426231312474293, 0.31738627446919121, 0.98052773662963755, 0.54906669154999588, 0.7306481551870585, 0.91122866732966346, 0.12487398339355948, 0.76713968230753071, 0.618707215915597, 0.91979922806146863, 0.64390860648760961, 0.31141990354233229, 0.074803837115609118, 0.43820449995686428], [0.05293474877975668, 0.59874036347260573, 0.994730641275995, 0.23447163280404404, 0.081396131184527931, 0.22087379573971455, 0.963290508159028, 0.12286478403754542, 0.24899941617228583, 0.49799881271520341, 0.92898345288182749, 0.19207651745115994, 0.68267454084659507, 0.86557144183928347, 0.90433696334019287, 0.37294302319189276, 0.76571224271880556, 0.013334003612952228, 0.11183198945545036, 0.51881014778317891, 0.0004569537528418266, 0.14929425627582216, 0.17604121462071998, 0.64140260376639169, 0.28722216694833014, 0.49685988558224614, 0.67450990136140032, 0.19602575084654761, 0.67463485092073072, 0.14991470207922175], [0.603074291646202, 0.34215596272192583, 0.52040384759419211, 0.708474187158181, 0.0857255752807089, 0.91895189057390581, 0.91654722276353917, 0.69666027147459, 0.96479285559991146, 0.46994602307975308, 0.17940143420889043, 0.62508628933710408, 0.74413130365693714, 0.027151439601572402, 0.18227555661983275, 0.70134448008259032, 0.72616662825854039, 0.522046834532456, 0.31740141159713831, 0.61514069401297722, 0.0830606424543997, 0.629377632268738, 0.91387106119805128, 0.52872157297721034, 0.032926999696707537, 0.56943910885471549, 0.0095406552024532854, 0.3885919308106236, 0.25393991275602468, 0.89548319576965973], [0.47742011047200494, 0.9411209293277063, 0.1623431818122556, 0.57511707363094466, 0.64085594261605616, 0.38248756890809066, 0.704780431302352, 0.73803911149841284, 0.67685448933481718, 0.40251757330928051, 0.871032010348747, 0.30718307154211122, 0.30318136381722716, 0.49231521440410264, 0.22015750745550089, 0.35631904081039811, 0.47241232668956457, 0.77719493447143129, 0.33120762102615009, 0.44438584444104556, 0.054899540037757433, 0.30927004806412484, 0.23908239680883692, 0.36530566176635471, 0.78265636533937477, 0.708156078984015, 0.59677746753457339, 0.72175914552540033, 0.04745325032183878, 0.27359901496901207], [0.51717377791739383, 0.48237525655688296, 0.4989554469046531, 0.21043984552260597, 0.79069051212126851, 0.21401640277072165, 0.77252353213621916, 0.4041778558213156, 0.44953459219249192, 0.994316863228451, 0.091335689766844586, 0.26449793584983772, 0.45494455571465897, 0.1043581737903454, 0.93378046022119854, 0.51447727769773455, 0.62074085068850993, 0.47734207063856937, 0.43460752883904541, 0.44478937150257136, 0.75465796462090851, 0.53634846546186754, 0.697113005313418, 0.075074684970783845, 0.39834844899092925, 0.4927417479093652, 0.733674968503001, 0.86874831448750933, 0.47761299170478766, 0.39651601028638828], [0.32324410987579033, 0.13286740310971779, 0.6363241990528361, 0.23084601827524021, 0.6769330461494496, 0.0012818496596150863, 0.020676448279249193, 0.12533702730569662, 0.43663733031146312, 0.94714392784248791, 0.80480767798254416, 0.32594386545138976, 0.19666008862934847, 0.0999537937411834, 0.80450387408360757, 0.90425549248837023, 0.43460113064309625, 0.25266930375212981, 0.11260963324294726, 0.3148181981311291, 0.46531075160236068, 0.33601059966469848, 0.80793402700776029, 0.81639726014544056, 0.79049722928841037, 0.30994078480974552, 0.43885356661700503, 0.46155303641028744, 0.0066162080233014287, 0.87640608429899813], [0.16347908743370343, 0.82863441046388808, 0.25495927017780229, 0.15277043971102222, 0.14016667598422661, 0.92126898102274069, 0.050397781226727556, 0.46198066826227024, 0.00531771922674662, 0.69265574003552233, 0.20902782496421324, 0.69335035532955358, 0.04982629297481711, 0.4654429660836723, 0.510000053574241, 0.26695178882617565, 0.011196687721483523, 0.11270000143751002, 0.67310839245181631, 0.59495722781377347, 0.23526813710738237, 0.24146894051237078, 0.21125003669733478, 0.074871703605844275, 0.32964718580770425, 0.77760795030313856, 0.72565815612454987, 0.0724522405099296, 0.33036272564253433, 0.28911538197360531]] [{"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}] +20 [[0.684257829378934, 0.70388618730619446, 0.91669107239388337, 0.097070795145535627, 0.42518446096791107, 0.32749456014199685, 0.17573416201290015, 0.97077480892428736, 0.22136291388316398, 0.578723728596944, 0.87961023616505507, 0.64923384344922963, 0.17114415678257822, 0.32973463962151495, 0.81838256249274288, 0.88938390395559119, 0.016497830529647928, 0.43662933626524647, 0.70965984643835534, 0.44093262810597977, 0.054137428707956681, 0.075643253405023758, 0.83884039021903234, 0.48421763807034079, 0.71558418601253371, 0.80080476060084971, 0.40895931615416192, 0.16088512152326329, 0.512977371917522, 0.3160119616854663], [0.81306053344422713, 0.6322334642928813, 0.21247808463565032, 0.78357908124712439, 0.5976780287131096, 0.94479364139108335, 0.293402462909365, 0.60165572758977282, 0.97292634462827332, 0.608694274025095, 0.9787592554665383, 0.97452457245866653, 0.34286320304789053, 0.20158625315070788, 0.38165607394819356, 0.27263346016266221, 0.47920176529425895, 0.49937788664277916, 0.602077346789228, 0.13332110815448361, 0.64211402046436361, 0.60475846517080167, 0.92982618099858, 0.48854527817089943, 0.16378080450411325, 0.29546478418459932, 0.66196160881014632, 0.22252543622022014, 0.75350799299380133, 0.34647108880502431], [0.42685148051589594, 0.96006269938424182, 0.057642583526824609, 0.8937603749522689, 0.32200023400030153, 0.56540132741262372, 0.813186257077971, 0.0020607948815509758, 0.61910044711105006, 0.29184808292388476, 0.27091825240390865, 0.47490785004847624, 0.26729540466476842, 0.16396712948538539, 0.43200358606436717, 0.60261951447006967, 0.93442924198166954, 0.097019268564319083, 0.041475894127053325, 0.98224808803456742, 0.866100726904561, 0.46241681384594147, 0.21264981278387085, 0.76329891000290739, 0.41362492216945212, 0.29331601625089487, 0.018429804089057811, 0.67398285799902047, 0.68454689939623581, 0.09107091870250128], [0.37864853764453854, 0.96499617635454893, 0.403140180911294, 0.24112443558816066, 0.096980238171250788, 0.97129732173329941, 0.73336193107586489, 0.37601325994404866, 0.95663956764463864, 0.33707753951723041, 0.51302579347244093, 0.63351534545960053, 0.37099015868953789, 0.075098375878772, 0.075492478859039869, 0.649570208593355, 0.20335131191592548, 0.53879545170655263, 0.59396715519145882, 0.29845331232738515, 0.019704900198906916, 0.074025021005791, 0.67353201598651669, 0.64678838372401082, 0.21866937803459208, 0.58705602189814732, 0.73733091476191759, 0.32333201680011259, 0.19502109811402812, 0.51638678439856267], [0.60586266289748381, 0.523067090294932, 0.93620757235915, 0.72819265827760149, 0.55212829064626134, 0.16199295778747713, 0.22571140564280612, 0.66868670388186879, 0.030416473176450465, 0.2781018946857361, 0.93278761568790336, 0.70140254356106035, 0.22280102149269343, 0.63438289796068414, 0.70501088750510388, 0.93338514533242389, 0.5249625668914828, 0.621295376813799, 0.7482338445504918, 0.9915015843925834, 0.10173635218603483, 0.97900699278232062, 0.27797132536085234, 0.14856035771390796, 0.68282659677377766, 0.56379148886928965, 0.65688899335932893, 0.91865482085412753, 0.84041731187719071, 0.36833349111224023]] [{"4wNll1-xGgn5zl-3b9-Js3h":16418, "TkepGM-6zRAxkv-VRw-Q4aK":94366, "KVKuDx-5fekX4O-XlD-dxE6":27789, "4IhvxO-1SzMx79-6Kr-ZZ4B":2514, "nEwJeB-7Gl3iB3-my3-NMfo":32437, "BU5VhS-1VQynVx-4GG-B5Bw":95501, "H6xGTQ-0R5cYCC-1iD-LO74":95422}, {"4wNll1-xGgn5zl-3b9-Js3h":16418, "TkepGM-6zRAxkv-VRw-Q4aK":94366, "KVKuDx-5fekX4O-XlD-dxE6":27789, "4IhvxO-1SzMx79-6Kr-ZZ4B":2514, "nEwJeB-7Gl3iB3-my3-NMfo":32437, "BU5VhS-1VQynVx-4GG-B5Bw":95501, "H6xGTQ-0R5cYCC-1iD-LO74":95422}, {"4wNll1-xGgn5zl-3b9-Js3h":16418, "TkepGM-6zRAxkv-VRw-Q4aK":94366, "KVKuDx-5fekX4O-XlD-dxE6":27789, "4IhvxO-1SzMx79-6Kr-ZZ4B":2514, "nEwJeB-7Gl3iB3-my3-NMfo":32437, "BU5VhS-1VQynVx-4GG-B5Bw":95501, "H6xGTQ-0R5cYCC-1iD-LO74":95422}] +21 [[0.6282416298019271, 0.86423479871855158, 0.64051640054198133, 0.47154464676809205, 0.22303068332930998, 0.31937323748612734, 0.56317897125316452, 0.25979315710605622, 0.14789328251911638, 0.26376910076159166, 0.40027622345858727, 0.70243539501977659, 0.98536253767324122, 0.82519130249684947, 0.35414858262751348, 0.6894238443029882, 0.093520836477473668, 0.78103127732292077, 0.97859265893645231, 0.02078030359522598, 0.49122472672000439, 0.11970785905635695, 0.26029733668637822, 0.87121649622366948, 0.00987485616436512, 0.9452851742025481, 0.36314161781160315, 0.78151600114070763, 0.26339756075910892, 0.91588597870031863], [0.93711834637182, 0.3775132414333876, 0.66486215121134973, 0.96605287307654164, 0.14853023608370153, 0.28020116251981186, 0.80116553019631909, 0.27903036756071564, 0.74923800263843732, 0.089444711019594614, 0.96647414867264214, 0.347011044786007, 0.57012284020880721, 0.89897157213494616, 0.55062076628953149, 0.4857093069925712, 0.79755842258645682, 0.53175841448450578, 0.28041097138186966, 0.35419645675568989, 0.27157039815586781, 0.57262462816521964, 0.019803147546250788, 0.87175142830743779, 0.83049422282467777, 0.31879215504049763, 0.94918249153744649, 0.42055161544818587, 0.41932485296844813, 0.96936449307543981], [0.46445778635382695, 0.376348349085758, 0.27758479315000906, 0.73112364682219255, 0.06346787081075278, 0.81884234804889611, 0.02741964989153467, 0.030739039871592877, 0.82019167722019026, 0.739294530091044, 0.89376329405911459, 0.69242770887066, 0.78651453575247321, 0.18497779019765304, 0.60579747382531945, 0.76530013473365777, 0.8068104125596, 0.55839568573179521, 0.752563560629487, 0.66607223565652529, 0.78235797313141675, 0.72376478512783, 0.49546678383728038, 0.44051804042267451, 0.38061632271804957, 0.664076727114492, 0.576546441759269, 0.24098419233754775, 0.54864352494916868, 0.17912749630498559], [0.50376803483459054, 0.56748175676439983, 0.96044937085593962, 0.51822081650273988, 0.64322637732353216, 0.426316827066875, 0.70483156031431538, 0.90082351621953916, 0.70303118057875758, 0.32291607515588994, 0.10325186881001525, 0.59427054947880986, 0.75477992850084863, 0.92444940070160253, 0.53656725607365885, 0.58968801103300661, 0.50590532579278036, 0.30304670891192642, 0.55493844140972881, 0.94562264245697025, 0.0098171904877336624, 0.90185084229468115, 0.3469959983057106, 0.55343669944455587, 0.38064229290607909, 0.75384514811555658, 0.85235240222757336, 0.41934726850472603, 0.36549742287518483, 0.027338146432063337], [0.41580589734420359, 0.55505945134656842, 0.81961471041716616, 0.67218377377521588, 0.90208976707286914, 0.22226339676962148, 0.22827025158799108, 0.79726127097541788, 0.6167787935153618, 0.58583511837145341, 0.42435051507918276, 0.42243308111838085, 0.21197223957190403, 0.7964013604241047, 0.56152353179187608, 0.41714555243798113, 0.99463049374802381, 0.1449995532617534, 0.22036460154813964, 0.76303200130963778, 0.70095037279216754, 0.21088373275463257, 0.24841929272437535, 0.75932267484755733, 0.85192207072577786, 0.85809746741869153, 0.56506920622518075, 0.42756258282724557, 0.263218129494871, 0.52725614608944349]] [{"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}] +22 [[0.86422759673630334, 0.92679339642157155, 0.039232949953334928, 0.36945170719753029, 0.10445499958631888, 0.6693224249600398, 0.73125389499857574, 0.75744906617478513, 0.71545217427979924, 0.805416660525261, 0.89630901914123173, 0.71948614477196726, 0.93691749542168989, 0.67053738008471653, 0.62821235526745189, 0.89034704938005937, 0.56893543616380793, 0.73804662755730976, 0.96969112113386746, 0.31105666569182355, 0.08005316545771668, 0.83085210167034584, 0.63359136988200426, 0.62968064023686565, 0.42734081046608352, 0.76956128031931081, 0.0209216397085068, 0.37488063142077233, 0.768153467069212, 0.40566140451337573], [0.33312799118303016, 0.35669801202424678, 0.2364769512392908, 0.77046887648246265, 0.20978371145701358, 0.29689189587181308, 0.48115340111003735, 0.63406806860461229, 0.98791526698725718, 0.12025822906936579, 0.71894726815194143, 0.11318595442127843, 0.37547576573936137, 0.94870887498061351, 0.64232529214203449, 0.092925762474396212, 0.059015698039078357, 0.73149012192975282, 0.63998958309159992, 0.13450930276689155, 0.52439029747867316, 0.61777327912638236, 0.43761562915212482, 0.22965408194558845, 0.15938169651995826, 0.63194962181173242, 0.44691184504341364, 0.23193811975661049, 0.52840275661339331, 0.94731530699985944], [0.82303138422101729, 0.046481731067986587, 0.56198950498846534, 0.070432032690099788, 0.47353040452644, 0.69150226435611728, 0.287576903683485, 0.546301803597882, 0.036481768396258052, 0.15877417375630598, 0.64309274082595425, 0.3735951123717961, 0.15101683105489261, 0.59475344362915217, 0.86119308906096514, 0.17952353112140396, 0.18048177517783759, 0.70010208837915067, 0.88950173498820906, 0.34554964213432826, 0.91655981215967952, 0.40257485835146645, 0.32100533591508984, 0.69797800330235826, 0.69212377634289368, 0.79670667814962659, 0.34879503130374911, 0.081573467940392774, 0.761248169591419, 0.063466367410725311], [0.33063387068701855, 0.11970372646243921, 0.7667236918810777, 0.584018311126053, 0.65136943118594592, 0.73110523837403241, 0.60858135987695661, 0.74396061103275657, 0.70499895881252794, 0.52122269270996457, 0.051547305863129167, 0.98788455012400533, 0.98323434650262143, 0.99517435129100673, 0.96501357680694977, 0.52882699691363111, 0.45370266711586182, 0.64974663330176541, 0.17602383861009174, 0.743163934589438, 0.638457374461502, 0.39342299905257927, 0.53393793478525708, 0.68871944378339145, 0.062264522371997866, 0.4163040489396217, 0.93485214582003984, 0.37234180570211528, 0.030166700793004653, 0.58587686158830632], [0.73451675376014947, 0.3629241680827231, 0.01511748629473908, 0.67994360698457457, 0.19040663266800828, 0.096824231285298157, 0.28901713333001955, 0.28462059792734917, 0.31676420860124077, 0.754863937269606, 0.81427040271242823, 0.98317075099528051, 0.81084848450399138, 0.90356649315426707, 0.82802555853068194, 0.54053429056312319, 0.5142879075345429, 0.18801481466692238, 0.9975431807265, 0.1487731200243203, 0.21034880297580372, 0.239692171127886, 0.50268240697696009, 0.44024722425880569, 0.42609516331063912, 0.61989178211515383, 0.91028102607686034, 0.89819166033152853, 0.3078297781928927, 0.057759472600305117], [0.24524525567117861, 0.49846525051606372, 0.56887057387244688, 0.46175951121030345, 0.23445390255292298, 0.9015932430150958, 0.97063021008641293, 0.18433046191739422, 0.3021206837767263, 0.28082775423607109, 0.791131056177174, 0.80172443842317853, 0.52904751332254885, 0.91980132004222093, 0.25950157663687567, 0.23449993531255453, 0.45324084490119443, 0.042450928280596933, 0.99066454015700622, 0.37886442233074935, 0.28565196166597862, 0.94824060354341833, 0.041340939772944174, 0.54504006503307656, 0.842341029030388, 0.41731311169191165, 0.326658548429516, 0.90605222021210585, 0.2531545672515183, 0.52305220399471131], [0.7860668493597851, 0.848816805903052, 0.29704402901094373, 0.44363874081709842, 0.86792517421613757, 0.58912926340540506, 0.035442811175551481, 0.59953550272445766, 0.61404464783925339, 0.66313479145103682, 0.26765827759399241, 0.46043707887745366, 0.86862813918731141, 0.94008543968710745, 0.21602375924377126, 0.66307279948826914, 0.91046729868741982, 0.61466651168279607, 0.76359533896960019, 0.66035204226056188, 0.11291897268386697, 0.2319017718518287, 0.51346642100394135, 0.98269327131909112, 0.073098074196493612, 0.17362226865932928, 0.91631335572597938, 0.33397916241585879, 0.019887703460604533, 0.19271184231513572], [0.72157286501585816, 0.16743220143951654, 0.20978120339490547, 0.51981714154476266, 0.047692713166035849, 0.090806277302752592, 0.45023521716035164, 0.78693823451942435, 0.43818479373798391, 0.43564194738817585, 0.85212695717363707, 0.5132910475023732, 0.37274988861589253, 0.59193380500760051, 0.40607249930110345, 0.18106044446223668, 0.83653152517629081, 0.25741082321411513, 0.0017772639819795266, 0.99213853988526579, 0.7155961872422133, 0.57973124563267353, 0.065254118150196883, 0.24167031706031572, 0.98302574939866139, 0.280246074318359, 0.075658087291649134, 0.87020400541687182, 0.86754710104496635, 0.23958580132631258], [0.841651234323355, 0.1634713936135328, 0.45541323802838185, 0.69817739546805446, 0.56662491896276346, 0.75729905826998412, 0.46628892548722722, 0.17889966369985921, 0.41277102209597283, 0.074878247433725509, 0.26518298657878392, 0.47443689489090934, 0.48590759735886213, 0.074598053509818851, 0.52500883981523694, 0.5584920000615945, 0.906128467267314, 0.71887229076012948, 0.33814349835599355, 0.077872245547963459, 0.90376066278344735, 0.88786812292904438, 0.99039437772516037, 0.16156201634991829, 0.68577689757377081, 0.55677006297121689, 0.057072367794676615, 0.64324871042557208, 0.15472555594277382, 0.93118997611523713], [0.95840903832214175, 0.43536926975703194, 0.430403322250562, 0.89256053536868962, 0.62034422375322784, 0.71559680529637848, 0.523910879457945, 0.54382547489793021, 0.42674982246707327, 0.79078760147477567, 0.44439518260718314, 0.86546725627892351, 0.22044451100825935, 0.98876145715351016, 0.480696871668852, 0.49288869340229291, 0.71289327309331885, 0.69488867628937012, 0.39328428474217891, 0.40839214741909491, 0.29627438654167126, 0.69830928020674454, 0.77627069273262239, 0.90447625578977109, 0.91663608026029064, 0.34171180493051734, 0.90497470252706513, 0.51989661881109317, 0.36315356065964988, 0.5789497158861695]] [{"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}] +23 [[0.014350715683691062, 0.31198088124604328, 0.44919313540445149, 0.69403977782546622, 0.87170873297793683, 0.66908538287752817, 0.92408997060257614, 0.040486850653616457, 0.9018635145076721, 0.0769809553877272, 0.6379446662337267, 0.3287831700120436, 0.54633922249848688, 0.37187021408335308, 0.46124694626748486, 0.26703904141878221, 0.074211030034582182, 0.28004057053656206, 0.35419094952035479, 0.0611746267096257, 0.15039656641669574, 0.35241087496641776, 0.87091663563131516, 0.655690048743426, 0.4252383900625164, 0.63900437095912066, 0.56744240231480558, 0.31172800797857469, 0.009055027885049638, 0.80516072502532676], [0.64425221531667176, 0.53918333416015907, 0.4814716804509771, 0.43524522521595321, 0.26875802158795148, 0.89668378686629169, 0.77772399763046052, 0.46980845599732668, 0.54336646433206393, 0.32658683862800919, 0.892761466064651, 0.078595178647893893, 0.011636068226680618, 0.50170772142855913, 0.84885962119634462, 0.784496405052977, 0.92801970707433112, 0.1263139763491159, 0.78358299457110114, 0.32248392675513227, 0.25594430926225065, 0.25599223454606623, 0.33232390845787041, 0.54276193987254429, 0.50448775309556837, 0.479313334870065, 0.703274459344789, 0.64881589002654583, 0.065671113558733563, 0.23879201515861392], [0.97257248000802132, 0.1987332811463246, 0.67239133934080242, 0.6594578965984278, 0.16349449283887907, 0.59211761525201723, 0.96127849129794118, 0.00042479202066081356, 0.61595201933043942, 0.21511090824897638, 0.89526815215429234, 0.24816534564842063, 0.0011800208080797736, 0.02248611448865534, 0.97743851404921189, 0.4297689053594147, 0.091389039842109665, 0.09846596578413247, 0.77066889750780365, 0.94028041670728135, 0.055286058525905957, 0.65686997847235162, 0.22693428771683333, 0.55028844127461662, 0.61775083455617652, 0.016386764595307635, 0.72555854890279126, 0.7882509317926637, 0.89558834841162038, 0.31821410661359062], [0.52150662458084041, 0.076617680921092668, 0.10116013549156622, 0.14100456397137329, 0.51280381671674746, 0.34162834003161624, 0.59183027864920568, 0.711999525290495, 0.85042796670357323, 0.81373577087425031, 0.93554468585583184, 0.11343025763383863, 0.93452858684384066, 0.38892429457273858, 0.761495272786269, 0.73970947185415326, 0.76758431203696176, 0.23076300164114683, 0.1697028125436405, 0.2736326361441469, 0.46811803712397371, 0.25885449154401119, 0.38448599303768227, 0.61875298509121024, 0.98784704568021076, 0.79140986148762749, 0.15416634353835035, 0.55029010908854792, 0.72065311235516083, 0.88800080597899467]] [{"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}] +24 [[0.61184783339368509, 0.90726494717810446, 0.052551004524158063, 0.68531092865850463, 0.075755468884431854, 0.23243887205627478, 0.016006660882104895, 0.55489866095327578, 0.35372039824367252, 0.21556295384126856, 0.6549740705467787, 0.023472470202435036, 0.59044348569592109, 0.49133437377913369, 0.14329430604691828, 0.19984437254013632, 0.59585062365578012, 0.4471476134147464, 0.00507627405204647, 0.92821258175716037, 0.489854027217573, 0.69896669376449327, 0.001897647243347178, 0.36595913422938275, 0.22067223054372076, 0.60192743816314065, 0.575342607152971, 0.53442522254210534, 0.068556167014707681, 0.86772174994276641], [0.86346193568348206, 0.3722357098466762, 0.44305710539417176, 0.4711672727591244, 0.39665428026587823, 0.446720870385205, 0.50951156438126033, 0.41996791797408417, 0.19663240219672462, 0.88621008165500126, 0.43920722215859553, 0.63452145426464157, 0.15155144254304354, 0.28444860782758485, 0.36773319976999186, 0.13873293781487994, 0.22104077475690886, 0.29811544774116683, 0.46730675018572565, 0.99814347686572469, 0.23679509946317856, 0.40939629685862267, 0.89140766983918107, 0.95828565228061169, 0.80754416418043173, 0.85779277916454377, 0.73384093609018886, 0.53402885775548492, 0.11163475307662785, 0.058825334716302025], [0.559386849562951, 0.37977868726594954, 0.1661228975089587, 0.92201375316275991, 0.50948735280022583, 0.13074681650034548, 0.49818769808097907, 0.35649605477937674, 0.46395567135083093, 0.54710854463691816, 0.012843003147055332, 0.909872701998225, 0.24848652836307605, 0.71571766544974869, 0.25033647687220328, 0.92434013832638884, 0.41778186587660771, 0.26814910569761474, 0.15729018803863293, 0.81390356510356, 0.71539755520156822, 0.059191650184280187, 0.95755070241850959, 0.067436227847928465, 0.91395418359407221, 0.516514978993087, 0.70188088429268591, 0.36457820843832045, 0.37314540182784117, 0.92008583208863393], [0.60003925291313032, 0.75494502573079136, 0.31801523221235706, 0.19901093622015298, 0.29445737690556495, 0.19584237438657137, 0.53626578847805551, 0.44383496811119982, 0.82163953134096079, 0.86966015095681, 0.92089506912885821, 0.22392350492929813, 0.12557267194665045, 0.60934545474434942, 0.0432541922118278, 0.97241309155540723, 0.63684962255191624, 0.44883085137049827, 0.731652123676696, 0.15531120765858852, 0.61187142395067307, 0.78502037247831091, 0.45192513351289409, 0.67668141368847756, 0.63840975040328485, 0.88891310492860665, 0.13892737211786843, 0.988449538534481, 0.95697364731657208, 0.53566975735402167], [0.011687855449314721, 0.67121939157375166, 0.021839500863293115, 0.33690505102353496, 0.82001818377685964, 0.8315869807159525, 0.031266837552144322, 0.95321478307157348, 0.91664479329888826, 0.52179478307792182, 0.9933365952399702, 0.74092323313963782, 0.52647793260921183, 0.52916831217902072, 0.26201308560334, 0.54426290316104708, 0.59341914686376307, 0.3056293034913341, 0.3774439252966888, 0.28267573549539726, 0.625147978864336, 0.51168143616285544, 0.47517756361610075, 0.70265047329589636, 0.027617528696819549, 0.63461563765746709, 0.49089083558414481, 0.16448865767759757, 0.52352776495978437, 0.40728402102745453], [0.88038413835515, 0.61661699646567436, 0.38283429103645572, 0.30164277222703961, 0.59901492516589794, 0.9735419130629287, 0.35838974198496687, 0.20036616349234082, 0.64040500471769024, 0.9582667625993363, 0.41610348232060579, 0.46018508155788806, 0.52117082345381427, 0.095970891890699628, 0.57654505721604965, 0.9722053227533527, 0.71659001361003394, 0.95297437321138312, 0.44370490941675622, 0.49731215422780839, 0.20153177274627521, 0.90349004183731529, 0.286666368882732, 0.508869776699888, 0.931642281151314, 0.67639051125794625, 0.8612302936916908, 0.010275009748340391, 0.31169083726394564, 0.68519457168007014], [0.51107420285637506, 0.31152695948838116, 0.44114632716700986, 0.67670379857456531, 0.4718161535199219, 0.54682544657752807, 0.2014674297149337, 0.547196859892135, 0.71639265870663671, 0.79431267092405267, 0.055254711538245793, 0.26981554066701563, 0.12644925979350752, 0.019424387086474204, 0.76432574421613564, 0.75275327456739016, 0.25198125977569574, 0.7049177776770329, 0.10193978144804283, 0.10508302986358264, 0.43025118527350381, 0.9534957743132052, 0.03381127089033753, 0.22553835159543023, 0.57504876216779888, 0.053048165816900972, 0.77321656514571069, 0.84928816934723694, 0.51864804021491151, 0.97988196811505168], [0.57741164568974024, 0.18872173201159081, 0.3913554416317343, 0.081914855264381026, 0.14574304529094839, 0.086487379648751528, 0.15733180334665597, 0.2316572030036429, 0.0042480033148998286, 0.16250505131147908, 0.090409978707767236, 0.067909715269275028, 0.31720418277496709, 0.78139905093031337, 0.66991933318788111, 0.55545507762545943, 0.15300315383497387, 0.79277417420428153, 0.10406709603819286, 0.98362392248137021, 0.34988590172846257, 0.90398486771912157, 0.44721540172686947, 0.69746664562262584, 0.77649290734045828, 0.23352361741704009, 0.067432641907749669, 0.30936453366817529, 0.9628451607445907, 0.062881291772265535]] [{"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}] + +-- !sql -- +25 + +-- !sql_count -- +25 + +-- !sql_parquet_inner -- +0 [[0.2434200686509983, 0.24234602011681528, 0.64895448286354074, 0.80656526546697949, 0.33553233191049614, 0.93104045037487582, 0.46082051709757321, 0.53545409981555159, 0.11940161295457363, 0.50894827151143052, 0.82244549745752793, 0.6191370566860076, 0.33453321135998482, 0.075267042138438045, 0.40832636186584292, 0.19376042004217831, 0.46281496989081572, 0.54388378653102276, 0.39025393742796888, 0.39257363676724322, 0.70940372601174673, 0.89336874762060425, 0.35773689855687063, 0.46342745184481304, 0.82289964786839931, 0.11884358445222254, 0.73458918170034215, 0.16164855600390504, 0.7934128521328675, 0.76734686527974572], [0.21514872307532762, 0.4605570741471402, 0.069666541106819135, 0.70492028738367862, 0.73675974268412914, 0.40412098883943259, 0.57006914809817433, 0.81149134311857707, 0.029564542413484163, 0.48971676861864066, 0.42772759377026504, 0.62872940681697165, 0.84031514009422226, 0.31369342031813896, 0.10566261593964665, 0.5964036161859344, 0.5138291011545647, 0.33841236473208214, 0.24602629915282337, 0.40437292666699054, 0.7338776102306358, 0.54666339963226629, 0.37591059364035473, 0.16018983777385842, 0.87470842830327, 0.43783260127427237, 0.38237550446382929, 0.52640124030903857, 0.59507008866905564, 0.28992512490609956], [0.4906137706047905, 0.71698089895602046, 0.018517972540717631, 0.862522112228984, 0.33170845904568691, 0.25893021810096062, 0.086946580214511737, 0.74797772976901011, 0.57022889929989906, 0.72293517290282172, 0.44870686665639714, 0.39163874832166956, 0.47694942011079, 0.20606884449936869, 0.54833131681780367, 0.39659581979124714, 0.77826407943183673, 0.20988062448836364, 0.4218034645712998, 0.10445170058384445, 0.091410387129209791, 0.69881468554599568, 0.0694726906470935, 0.60081909208144668, 0.26340318020218456, 0.54675215456939352, 0.17812927994610817, 0.23273644898546708, 0.9716197201602389, 0.28357232035945179], [0.39441854927881315, 0.44491698100044685, 0.015265418569674116, 0.0389806629892302, 0.313687162692601, 0.24272318875770671, 0.82948922071003794, 0.84745207936479738, 0.84151880294739445, 0.87198460449683468, 0.88770647279663939, 0.025990169073353608, 0.609630015367714, 0.36925451158526956, 0.82272594449853576, 0.16550440916072373, 0.68346184886158567, 0.4808263638007455, 0.29356743977889554, 0.38695873606218245, 0.88571628064918251, 0.074808837828481245, 0.852394676517316, 0.90504678207366152, 0.67825081468092452, 0.57778337412379366, 0.32477763545864691, 0.61500027585520278, 0.69415865833439883, 0.15604596555016248]] [{"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}, {"V6ntMl-45ehz3D-46B-estX":82721, "aHJtNa-LIVgh3k-IAO-ahDh":39539, "x8ILIb-ZjOrZKm-Zcl-ZERz":42957, "CZ2fem-4A0m6JL-MrI-VdCN":67006, "d0OM1a-tCGRGGo-ddQ-VoM3":22143, "OGTl2j-NP7Gy4d-hZE-8BnV":99120, "LT7ZJe-hBCmLEW-HvS-QyGG":19978}] +1 [[0.441213983188219, 0.11829553460573472, 0.33551661881638117, 0.15769240977086962, 0.039896571828509142, 0.84339289915401416, 0.29772970512610364, 0.68175512882934242, 0.13552490402879558, 0.73721186924123583, 0.94628908401158074, 0.71997162119921032, 0.804720529599055, 0.1151729366872013, 0.22392404886137851, 0.68585234720029475, 0.59439613229450683, 0.44486208624901169, 0.34252653767716268, 0.8562974946251527, 0.201746488817938, 0.45932588358867166, 0.1517964539013732, 0.74892856020576515, 0.50058056193959433, 0.26948611062925332, 0.1620437436263531, 0.82698516140777634, 0.52889327911424766, 0.18066087004085785], [0.010362183096527811, 0.78999638908659553, 0.56527131322878821, 0.25118644305052973, 0.62521031091709156, 0.11956844128064659, 0.090671740839035064, 0.37039189414085794, 0.46650530859147266, 0.86701370327619787, 0.257308726078998, 0.7786868003157752, 0.996655771716326, 0.89317874309062573, 0.4678275506522982, 0.50427955961462256, 0.97252972900641865, 0.90071181183763371, 0.80564182248990168, 0.14905217075470756, 0.23917456987234886, 0.63514306375666063, 0.35125400292656417, 0.82037477409142678, 0.4531757782350474, 0.57353369037566448, 0.86957535401118757, 0.7761514363463532, 0.5668618982603183, 0.26808269266896068], [0.95970626710882678, 0.527145829825849, 0.96413320187391827, 0.26696331606503287, 0.52133380723923828, 0.723644164135481, 0.85451183333088321, 0.19579339800924433, 0.31084787004749748, 0.14276983054377468, 0.90408211548872564, 0.15052724676616536, 0.43547344603811033, 0.48827227916236604, 0.11030881668576298, 0.026188858674662363, 0.86516116929528719, 0.51637029608157869, 0.61229299640390633, 0.78574995645380874, 0.42967394974211215, 0.20923834812955489, 0.46884389485729805, 0.95958809593433414, 0.31225284206616133, 0.33198129009681343, 0.43381692834016017, 0.85834392136876247, 0.75144045036335017, 0.72945158090582629], [0.41148698318265997, 0.080628601445879244, 0.51908547785444936, 0.14602044641194123, 0.58429759017063732, 0.90191591696415441, 0.75524762491909725, 0.18442255638470018, 0.17114615516780385, 0.042086027519590341, 0.90966685818779669, 0.25905105300492437, 0.3964358877453904, 0.12647055730932322, 0.86465695137615584, 0.044588323690071596, 0.52844566281610728, 0.87888197979543636, 0.79572794446681483, 0.97261514452511433, 0.7801388914181987, 0.714992559374499, 0.96393816592887827, 0.78684880328993734, 0.45635336894181222, 0.94056108369434654, 0.42792083698815186, 0.031918723473152943, 0.97838382176316985, 0.20138883091457016], [0.61372663360504942, 0.62109593784511574, 0.32028937889960285, 0.18051830601541996, 0.78706809118228849, 0.30515011469959086, 0.95082890257741848, 0.79307166320945077, 0.73524739424480867, 0.94751668871728045, 0.59074667941030223, 0.29025835124102317, 0.38499449770058913, 0.00021293843938408052, 0.96720187581129224, 0.75146429874269172, 0.65763746918694732, 0.47045135031494989, 0.27545538564950856, 0.41696777864529277, 0.5944726751300099, 0.96965493750497422, 0.24249763244399558, 0.19371005821830178, 0.67289749618156347, 0.14458810683840095, 0.52812170975295925, 0.45268144370116903, 0.277813589420575, 0.90457374897433984], [0.696016178629173, 0.92886645345961327, 0.49300409159889391, 0.37927540045969554, 0.14077260370492817, 0.13332451337850637, 0.73363346056273027, 0.4682811018658859, 0.16627224956727493, 0.19186531412796781, 0.25371400000272659, 0.17673319196098103, 0.87673810618486026, 0.43591107023251707, 0.23132814824794479, 0.67050643427900936, 0.663807544560106, 0.38852024304634591, 0.959591497213437, 0.12994434759112683, 0.85130162363223127, 0.01218078584309612, 0.65278621627934919, 0.032921463646273241, 0.49393740979443912, 0.21486286764997298, 0.51479014418195534, 0.45600752373650089, 0.41705027196238231, 0.64695489457111821], [0.010872207050034333, 0.34676659181731817, 0.33146247292351982, 0.764088673357802, 0.73412006722706558, 0.49073268630840106, 0.260956053085703, 0.97949249370683855, 0.86421094587619129, 0.073360849983796328, 0.57937707834508434, 0.48439425146535342, 0.97893239094856344, 0.32026056938771863, 0.67021925505543234, 0.97612870645029548, 0.8978046076683911, 0.68827624281371669, 0.593605481730133, 0.10098668029759172, 0.31235157468558294, 0.47548726218341542, 0.17390522997606173, 0.89247584555755188, 0.18901337317876488, 0.51995793603416451, 0.60579719821566691, 0.86554667105663741, 0.31838727138203637, 0.86507535550525971], [0.017406020453996751, 0.71346870431074272, 0.92843114653706416, 0.88357146720574919, 0.42621258947549634, 0.74642642807099535, 0.97945680990576545, 0.37283257603708175, 0.17675784716142917, 0.30953543302073228, 0.61518766050226015, 0.5608583568725195, 0.25873006173059143, 0.42286377292471278, 0.25684945029677164, 0.79523948117202725, 0.86984586123985153, 0.79717510536860359, 0.99620794195376239, 0.92678112846734606, 0.21000613127643475, 0.19649359393185328, 0.26866407178466734, 0.04490732896058236, 0.771527256070553, 0.84222333593723842, 0.26896056090646392, 0.27373918121173457, 0.32481954332433061, 0.18343776065277106], [0.20975091405949442, 0.79574725634859478, 0.38970417939142965, 0.89408704578862963, 0.3172260304544634, 0.13413682986387454, 0.229630752888198, 0.40025512371111405, 0.40818926260404942, 0.9956322900379958, 0.30937762224383691, 0.58922972998760126, 0.99941638604690142, 0.65208444146382671, 0.716387856656251, 0.60703250864004221, 0.16283173582476596, 0.9223331219817742, 0.26171337555152741, 0.81218859959906331, 0.75897725934669646, 0.30501427350229693, 0.84861066455870615, 0.0208980781474305, 0.25644698283290523, 0.70850930050577476, 0.22515164059733495, 0.15240451247521969, 0.076403780549422429, 0.52599207032176831]] [{"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}, {"Sr6nM9-9OlwUrN-5DK-NZSi":38536, "qjmc36-SH7mu4g-3TI-FJSj":74853, "Qz2x5Y-Feu8E6W-PW9-5ZrS":81613, "uqorJs-57wbF5m-FWM-gsIF":36009, "A5JTCs-xA3WjYS-AkZ-9pZW":82680, "p04sGD-bR9gQeI-JMs-jYjZ":95977}] +2 [[0.76900112854951785, 0.040707996366250665, 0.73040821209474038, 0.251174093959617, 0.71938437935301336, 0.90000184400502226, 0.29695811719826615, 0.77132447239782309, 0.3364963599585733, 0.22297486715776116, 0.69454649562099435, 0.9551249988985, 0.96884572098791266, 0.17347093634523736, 0.56434207544207449, 0.19757550008890767, 0.44749957994579836, 0.43964797103244158, 0.95970263505113274, 0.47711000474054077, 0.91446112100589083, 0.54755561537061559, 0.21819692563011628, 0.98157558927278, 0.24414192846210792, 0.77242356958590552, 0.73240079339151265, 0.96731327956198487, 0.68125057384504861, 0.79954643820947857], [0.39562102911013253, 0.300589975109828, 0.94439596733490971, 0.5141902766602281, 0.89030184402086388, 0.74879475016033337, 0.45192941244060825, 0.55873485232275011, 0.65972125084670563, 0.23822721388911416, 0.5150274008892608, 0.58530492398326617, 0.35768484262264877, 0.97596885014058132, 0.31274217063022836, 0.45343252631214759, 0.80627742747477127, 0.54524341639399443, 0.21472498857621281, 0.97483191843791228, 0.0083334940950107184, 0.82683567249404533, 0.54955269146584784, 0.19115719296286693, 0.55913969284327059, 0.87355200182544934, 0.2300649781304791, 0.58565227093746508, 0.88098357508802561, 0.67037094227057459], [0.74184683416858221, 0.075891462031071222, 0.064515003540916216, 0.049845173679730626, 0.87696615308627079, 0.11234846570742962, 0.74248755966533075, 0.90048070712959, 0.2523041212873216, 0.71632024444791964, 0.34429919572017365, 0.20523359092311955, 0.090291964734466879, 0.830771387202834, 0.28385302713803429, 0.37792742214106712, 0.95959426901562117, 0.92895384894747579, 0.1186104612260156, 0.827310353904011, 0.174255834899384, 0.42819576403401982, 0.48431052587327328, 0.67163330427016377, 0.33205533710830282, 0.090042281031522009, 0.65768095432516138, 0.051045457970155139, 0.036285958057279855, 0.25297016086547586], [0.9285677775991299, 0.54076758076254783, 0.18990926998900715, 0.44018741039784814, 0.57326240959177788, 0.754917124803915, 0.37700575623008181, 0.61256779139616069, 0.11240590345885104, 0.34902351431470313, 0.75367136432443849, 0.095559214747875765, 0.70970229598083967, 0.278686201696273, 0.55898642693518308, 0.99894462297013653, 0.021774384972731431, 0.35249162808059331, 0.826409859339634, 0.769622058295028, 0.544846499403834, 0.82921986740909126, 0.12692762586608886, 0.37011069574867117, 0.52054077643672869, 0.052535467884380349, 0.20751182007155211, 0.45927775678209204, 0.76092502158510456, 0.89077093170390753], [0.20192255322140196, 0.8668211791466981, 0.33205129332030447, 0.05507597451328694, 0.68561165139634339, 0.59933114145982258, 0.89557676946990661, 0.20611887032922405, 0.22246401434815188, 0.59329324492697988, 0.64965891159075118, 0.12769091521900944, 0.20833846518166077, 0.439184806779809, 0.23316981120375879, 0.14497903900676123, 0.75484713502109013, 0.3408550784891925, 0.66137484175617511, 0.39593523123493291, 0.076538452134858326, 0.64566900012301431, 0.34657328471287241, 0.52608608154876657, 0.34562557769543734, 0.38343336851759424, 0.52379984119104384, 0.87028921982413388, 0.26062881597456455, 0.38591313715133158]] [{"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}, {"RfZmzX-vVHph7x-n0a-T9fR":92545, "fwLpqH-LqDCH4Q-6a3-fqSb":91323, "rl6RzK-n1BsJaT-t99-vh8I":81927, "SWtydF-iFAZ7r9-NNO-vRUu":14503}] +3 [[0.19201896280979724, 0.2277157594303113, 0.70696662663421062, 0.79613852089303372, 0.013154619183022187, 0.75111997454023149, 0.88382321644299544, 0.48943459157003355, 0.65226277171827529, 0.013413662629729872, 0.72326802798739775, 0.64237045403012538, 0.50375062994211028, 0.24899579433512853, 0.59781326778245125, 0.25528106793388783, 0.86173075992973081, 0.7486746409719387, 0.099299268038050448, 0.14850151396944167, 0.927790463055653, 0.0028725613850517639, 0.61728786534414593, 0.81819761160564253, 0.537622983127225, 0.61094125130920029, 0.60172562703852639, 0.87149398858891469, 0.71433858083551183, 0.29316580085759769], [0.49123087988432479, 0.34866866278473141, 0.25490758239544242, 0.198481216059325, 0.71511580179228973, 0.54975479458803378, 0.56106898112281323, 0.71362038603551881, 0.010263265208805161, 0.94870959378875108, 0.83717191918949041, 0.14835789294652491, 0.53323547998393184, 0.40038144179255375, 0.59480012365445789, 0.99670910532092361, 0.89349935355746291, 0.12382582530642205, 0.12766144391278778, 0.95538960556586761, 0.28541165853888295, 0.88009202376480189, 0.25010843602410593, 0.518535404496101, 0.35461442481978123, 0.353421537174577, 0.12878806551194355, 0.76162881474599053, 0.29874900496545542, 0.9918707222650498], [0.47930158736289308, 0.87777192682820793, 0.9729052229257763, 0.46668961209889614, 0.74838882464633516, 0.13589263595066881, 0.44697953455483241, 0.46935970588709519, 0.15703297491016721, 0.19316588128389867, 0.545124172609917, 0.78101350078908738, 0.6410163621688012, 0.11287592856305784, 0.386726865138738, 0.42102897746874313, 0.79283847003846908, 0.014142570868491022, 0.84386885813364443, 0.20991271398345579, 0.882542425059416, 0.011378082313593496, 0.13196624032401694, 0.26615522196368568, 0.91487698066175271, 0.98903826781794213, 0.45830797164952564, 0.79756295020852341, 0.46635555137147278, 0.80117489805872322], [0.76746156055211889, 0.04464106781443844, 0.7381555562161064, 0.68549383471345238, 0.14302482340606837, 0.38692219967019836, 0.85943425828802733, 0.038066621116226496, 0.3519608289695505, 0.43270586566689806, 0.980520942319257, 0.20754016648826257, 0.35075988224655175, 0.37849413859595671, 0.10657689719987318, 0.63663357651836272, 0.31193108009393855, 0.046479345312113285, 0.93969828894637586, 0.50390675553891506, 0.87483341624916122, 0.78675972974144137, 0.2210697010689755, 0.00023065065311922428, 0.87661405670073522, 0.86552932175105934, 0.19465852434769881, 0.51208470077970236, 0.77702964980771394, 0.05130887237216597], [0.850073837810945, 0.27775156946589707, 0.35518403819133615, 0.85835472931162415, 0.54459559773437716, 0.5326754255996724, 0.026723499549685004, 0.309702741762244, 0.45301312162045226, 0.76669468707848554, 0.30678889960312616, 0.14187528137346239, 0.048794797484269181, 0.4785560604824648, 0.52703128303907976, 0.089098475256317644, 0.48124329283070888, 0.73206112323895434, 0.9042557970716445, 0.25694889306889657, 0.28067359376324696, 0.893393269338407, 0.45376658932072966, 0.623667195113893, 0.297284510780336, 0.16061676229363742, 0.0422425514548026, 0.39194830607655395, 0.15927583233411913, 0.88403624434114714]] [{"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}, {"cXZOmA-DIrsP5o-rxX-mg8P":68605}] +4 [[0.59645537235209667, 0.60470950911235721, 0.90236107510871155, 0.66714780095213988, 0.95683682493491606, 0.0084652348887768181, 0.44903257208378411, 0.08657419553688217, 0.17840048948389087, 0.024768644008877794, 0.90679260937660666, 0.090295817453005744, 0.81267474389430649, 0.53225725844427452, 0.026670213937739828, 0.38148239227264857, 0.79269793518267984, 0.17868443924478761, 0.97028447888209113, 0.1496567720548948, 0.868275994476362, 0.27988042656300116, 0.47835050846162352, 0.07619909527486779, 0.71175422888807716, 0.638807768459914, 0.091001101168208254, 0.429455077477475, 0.81945551496522451, 0.81635799081909854], [0.25845611414342695, 0.93588080812337937, 0.54975845944769153, 0.54666909963684862, 0.599515977269857, 0.60291328537387323, 0.51085808764462026, 0.057126078723462981, 0.1405517516909176, 0.56476462877536227, 0.039572991518131762, 0.026839799812264697, 0.09910884735050518, 0.32409492347678981, 0.69713263993041563, 0.79728201439697854, 0.66393751595331463, 0.47052706818603907, 0.54869894274283593, 0.23488762718234768, 0.26720782241176821, 0.071224468160711929, 0.49044669171465083, 0.33781736901208692, 0.044862711179498138, 0.4492603646461405, 0.041347533684643167, 0.27770773470068966, 0.53750775889786506, 0.99226541237145072], [0.8263725295987735, 0.072117141469498436, 0.19693161240786194, 0.96684779209180416, 0.44364883344883, 0.49451042358532349, 0.93229631065503171, 0.0535381078552557, 0.53017612617495835, 0.14600645860097616, 0.75523242539775914, 0.46251916634464785, 0.61902877200653061, 0.83674681437105658, 0.56451851528597119, 0.042158274583507627, 0.815540183867558, 0.55462562167029039, 0.248720542423846, 0.75036780166853123, 0.41078590490136058, 0.33525396449642286, 0.57911156542625586, 0.22307758063375926, 0.2977524630546049, 0.289415268442888, 0.81538755260778861, 0.94461005454552083, 0.45889921113109666, 0.89290010087251326], [0.553140778444955, 0.70150761010622043, 0.85339078143198754, 0.39849248504695256, 0.10474699234992813, 0.584368818816282, 0.74809804480878206, 0.89101199561190281, 0.62426819902956632, 0.79150821255452342, 0.90853574748267352, 0.073699331422595771, 0.26216846589872445, 0.88376226554960446, 0.094225484142207327, 0.3478650714085475, 0.867084751299976, 0.94152179737902475, 0.98754263736534653, 0.12760503926495059, 0.13959090555373466, 0.2830789459171249, 0.025926563414585324, 0.61751781026145469, 0.68683762078068489, 0.017634678420189953, 0.60440356004121276, 0.47374298117945735, 0.12714672819981787, 0.726820587690857], [0.768692236624599, 0.86727171513467127, 0.25008380522776952, 0.99534657771489832, 0.82873131250276744, 0.47336375191149926, 0.25022765021774163, 0.34439378559597966, 0.28829068512851008, 0.82203788415633217, 0.22180125386949223, 0.36062986303979305, 0.10839585385643646, 0.2619857360401604, 0.42652382061133332, 0.45923190868591168, 0.6006171383174741, 0.51389169281847558, 0.32848864581573878, 0.77105944194000031, 0.18084506089320507, 0.41902416772213746, 0.027567250463190196, 0.10212846222502703, 0.804993201274559, 0.025096695934428759, 0.516785279349264, 0.77779481981211462, 0.15514080565785959, 0.59378001962267346], [0.13931096759109174, 0.45456305588268631, 0.54794734273235213, 0.46287887973702191, 0.72714235382191816, 0.65919531179500113, 0.0967414439562464, 0.95107024070960655, 0.16627495647351564, 0.89780128277916238, 0.87450744041810613, 0.65423286016217586, 0.552076437866004, 0.66684999799369915, 0.69706229920355778, 0.54978003184689039, 0.67379039427278609, 0.59354989215513521, 0.67403692534859727, 0.29904098633776333, 0.29853705384444806, 0.23480334473475006, 0.85984522310246847, 0.017618954535161491, 0.61287308656206507, 0.77194823983036887, 0.17764550411803548, 0.5697865206391971, 0.22391910519470948, 0.95275785041023753], [0.286368333266395, 0.13536189701183621, 0.32225775514574584, 0.15267755568543573, 0.21420587408132885, 0.39614265154377015, 0.96525156250005018, 0.35403516760008491, 0.39152755316908294, 0.19385604485780106, 0.952124029425394, 0.17496903468384561, 0.18200771594984544, 0.92271789224700884, 0.43109376824244017, 0.81907650123635234, 0.13282079376547473, 0.23707518579698394, 0.20404580115137794, 0.96229214910741256, 0.94308004342584906, 0.87823964362480833, 0.67567042589343407, 0.14526417408333436, 0.58426632020867664, 0.55450295032128571, 0.53169462264897926, 0.19750431518208511, 0.097213335953783675, 0.84445706310047719]] [{"LaLAYo-uI4vtyB-0Hp-hHdZ":33873}, {"LaLAYo-uI4vtyB-0Hp-hHdZ":33873}, {"LaLAYo-uI4vtyB-0Hp-hHdZ":33873}, {"LaLAYo-uI4vtyB-0Hp-hHdZ":33873}] +5 [[0.21999856488055747, 0.80106949963414387, 0.37888818235979371, 0.35621115078552767, 0.25905903110594364, 0.071486484085366175, 0.0873221357781595, 0.67747056809382589, 0.82675146444177572, 0.85778986301193283, 0.30961240678283997, 0.53293854613108982, 0.68386730354734, 0.7955796080221551, 0.78574708007062621, 0.28047617531729108, 0.020318084552748306, 0.311024599080163, 0.60445405220301629, 0.32359007828189013, 0.37264965333025513, 0.23379841929490885, 0.6143202190477125, 0.18205793005748117, 0.77837756124246793, 0.53844539297614635, 0.10183779363565648, 0.92238378788141, 0.87701308106567466, 0.01608737005588412], [0.827594077243545, 0.041497692464448455, 0.90690692210186918, 0.3004520621273975, 0.3292052136818755, 0.222308971558546, 0.19254566802257933, 0.95052386660978194, 0.12322597847003336, 0.86116851036162745, 0.76869393951040421, 0.22668843353210077, 0.068151646614525774, 0.9049898506726346, 0.24578262572798726, 0.78806300236832993, 0.68675973630725318, 0.32972487495192571, 0.885201873176727, 0.78577375258577131, 0.68842305374817314, 0.54208504069630747, 0.12366506456834347, 0.73026336262980807, 0.28065786702933149, 0.092844086938057124, 0.46256759838651451, 0.070280518678416, 0.1140538239204173, 0.35275389620574016], [0.72127280083399714, 0.6370248727762744, 0.86586544417322375, 0.22105031652880713, 0.14251426244787346, 0.97899028726172255, 0.53492708209058359, 0.51172091448127688, 0.73361228695521608, 0.14137020164834346, 0.49418706040976457, 0.39415230377827759, 0.52895982416633347, 0.62845853477625957, 0.22703849674065735, 0.15246962286528309, 0.16724085771160924, 0.12830084815246567, 0.45975791906683117, 0.57182621567457637, 0.29110243257535884, 0.075919701455123056, 0.094745057325779536, 0.46009708072614386, 0.79750880184650041, 0.28475920679372346, 0.947890468809938, 0.16987453936977426, 0.89376440953630976, 0.68556400758291], [0.79827990931672288, 0.49836024981190596, 0.24720026560360819, 0.67772774763967436, 0.66957780499577813, 0.01169281854023474, 0.25577548695998575, 0.72021704646774687, 0.57981448711366768, 0.66544559481048049, 0.61235444546351325, 0.56749680757393017, 0.21993494573022532, 0.57447480679533891, 0.88717707071425911, 0.99045338462680443, 0.15134942725446887, 0.92299290712958282, 0.81104803934436365, 0.91313197188281259, 0.14885949671895526, 0.29968797178357864, 0.30602835333328526, 0.64317558222495907, 0.25771519672292986, 0.19290497448364763, 0.0809204805995235, 0.2771724312750643, 0.98219271608465208, 0.98735000587042643], [0.78559599543873571, 0.42168973760716277, 0.77796388152736951, 0.566474052047993, 0.30122309543997206, 0.0865915690494371, 0.6816622920732831, 0.54363806097012024, 0.73270801172402322, 0.97930896151359914, 0.47843333112532027, 0.22035642249134402, 0.065173417541513623, 0.45705103685026793, 0.2818881950006954, 0.6013615049668, 0.24805379747880352, 0.078515713454836988, 0.78076499210593542, 0.1839264551324068, 0.67839597182795874, 0.037479387461347824, 0.6927327002951803, 0.78283004396700673, 0.66921098365878673, 0.75771478581566709, 0.1581564553425886, 0.35254123546476623, 0.50923058982698488, 0.58200694776128936]] [{"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}, {"xOo4P7-6uye7Wi-7KQ-wvB9":66996, "cytJm2-oxcsTTm-0DR-EchE":72491, "Fcl7Fp-7FP2yq2-hzQ-7oyW":92008, "c8kDWn-T8Zklek-TWS-zs26":35117, "900HmS-x9NfvV5-6Dk-0KYh":22888}] +6 [[0.74898892346933432, 0.0772671311088804, 0.7484759208893712, 0.5151660040344227, 0.54793263204154052, 0.17860231571322482, 0.45191545547845136, 0.23965009227204648, 0.75616725349983827, 0.65085678938875624, 0.091046660666927348, 0.9603655949205443, 0.22065287928589716, 0.27452040044972426, 0.3514822877984074, 0.34117875302795775, 0.097458775756514959, 0.58830642952170442, 0.73846443555926278, 0.55919580552106707, 0.45037858572707812, 0.13647543794934003, 0.13255920726058945, 0.16871606874387091, 0.12323980217004848, 0.11077143650340115, 0.98716376144590612, 0.39502978088772089, 0.14875498433404555, 0.87662506786729366], [0.097177540615967284, 0.58062553328546673, 0.32883321513429076, 0.44066079247933532, 0.94338469633061894, 0.90187984272994726, 0.649590262283012, 0.39017853733540808, 0.84854017149809147, 0.27605888713519044, 0.87886616028454245, 0.21516276554977509, 0.5688472393701216, 0.586471076399407, 0.034879595323582646, 0.20950450881237637, 0.48845963793495772, 0.96786578730492157, 0.41171826872655148, 0.34050533532799165, 0.28483627140014423, 0.91468253233487573, 0.50068527422610676, 0.57436437151025421, 0.78922278893976228, 0.40151185527236943, 0.45609134864401957, 0.15023274081738092, 0.77922838014926521, 0.038137361409268711], [0.71990374589138273, 0.20655242892429937, 0.60941270448373053, 0.52078629013512845, 0.041731907170210247, 0.98077942068912782, 0.9109919617597787, 0.28672320734582168, 0.40895146831276097, 0.52722383207909718, 0.073219585257285558, 0.33108023790161634, 0.33633774792525339, 0.52762248565689507, 0.85669150222179846, 0.60282387017136729, 0.5554001710037908, 0.30519204080002893, 0.2827714878860903, 0.26101724567386131, 0.3906799976541605, 0.59529420909701591, 0.89352772091248334, 0.34304383954521367, 0.013593233816938666, 0.97447649635850464, 0.58301645554604042, 0.073216969289620626, 0.32397159130030639, 0.1045354893461603], [0.463444487297172, 0.12320660017285612, 0.42527426005426394, 0.10938220584363356, 0.64868713571057757, 0.62497350043169841, 0.53151368070810157, 0.46987168786856615, 0.41283387260934545, 0.63240813677967611, 0.67225902780435809, 0.79539228668720152, 0.91287911459943283, 0.029629459420203452, 0.74665101864135364, 0.71089222140789965, 0.55237067224930125, 0.5919070102404338, 0.31818226048488074, 0.88415471720921213, 0.1067743357336548, 0.019427107936906385, 0.0390677907616207, 0.910514798225775, 0.93150749994813453, 0.28463188844732745, 0.74635025896396823, 0.37084455767769875, 0.930040809814289, 0.560617829063112], [0.081455059011764974, 0.88351592241115606, 0.47216182668981466, 0.3310881195792611, 0.95073079898694557, 0.38218296970644983, 0.12143973746040382, 0.02161107241801552, 0.2665807749589495, 0.16860874262718806, 0.85612613249317748, 0.63744452192773826, 0.18711140333684173, 0.513394186725647, 0.44917145318950447, 0.44494417883418247, 0.50679418964395673, 0.707349534003552, 0.81073247699139583, 0.69158147875312181, 0.941457489312688, 0.50238175589847733, 0.39049146863822426, 0.723854279214668, 0.79799549183120433, 0.8231538671891494, 0.187302679262491, 0.63288189230629055, 0.92250126998609983, 0.47940516696581537], [0.52012767322904718, 0.73423333183112993, 0.88010188280814639, 0.36370414815398211, 0.47691547990367278, 0.77154578001562579, 0.035592991776508764, 0.28053841936003565, 0.8770300969408179, 0.72274788758822017, 0.57604170193939641, 0.12002936668819641, 0.6580216746373948, 0.48321040225048628, 0.54484563267470509, 0.56319130779392468, 0.828984464752151, 0.50190461026070765, 0.64130752055162077, 0.88553844661451409, 0.52043883831107252, 0.65465648859426762, 0.880604218971964, 0.71197519373258966, 0.20462071644749458, 0.25284276124900629, 0.20609954268305353, 0.76828857955440211, 0.86025657642428566, 0.10638234027949556], [0.79927111579241272, 0.77319412491072026, 0.71234786291599161, 0.98663510054265424, 0.142090640055615, 0.61976951058111007, 0.190183706978577, 0.66782683020346556, 0.85251866535275045, 0.90624277973045653, 0.915746300478831, 0.96487824207406669, 0.93584420334129326, 0.69348535876560369, 0.85986857055398813, 0.83102240017404283, 0.37292014826736763, 0.090192240263251344, 0.28494894692465034, 0.63035389280036469, 0.90091614453135149, 0.8966148738932973, 0.73072374474757407, 0.15845988891742246, 0.43303037476774986, 0.073845212252012016, 0.093542343140423978, 0.84854211055314921, 0.81067206111666534, 0.24803174961051633], [0.79978642793326571, 0.87094552308766038, 0.48494745484892188, 0.80754165858400517, 0.18051909146797973, 0.51076785353183352, 0.28385117255923664, 0.40596582722336927, 0.41714567162077, 0.2912322251847409, 0.63738659972639689, 0.84499601014539483, 0.84681275232183539, 0.79976200863446456, 0.504907913103317, 0.37858687746318287, 0.9914677636257816, 0.069205935699441334, 0.77729751106696932, 0.453447673036537, 0.40594647334095058, 0.66841840246964512, 0.11116279885676761, 0.78079189129873006, 0.32815105308790737, 0.36440839558660265, 0.99305629233320336, 0.42040545216651659, 0.1583268525896967, 0.33252772147316867]] [{"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}, {"l3uYZE-WRdc5dz-pCS-0IVP":72625}] +7 [[0.81016551924298819, 0.034827311984228504, 0.42584577044060956, 0.65761719473752112, 0.015828067858713157, 0.77104072310432981, 0.32378500814571542, 0.96337227876248355, 0.72757792965145829, 0.5567974933569082, 0.36826830177176606, 0.38974273844483154, 0.57162102356777855, 0.058465386858290835, 0.91864498625008917, 0.31216011334835558, 0.6550489555175163, 0.625780920785453, 0.43187750880489983, 0.83676673245429811, 0.56779250439812456, 0.9354600086277306, 0.62236094937422193, 0.15691930283260036, 0.40231063331435424, 0.899866096151119, 0.00014995422647023915, 0.36562233778094633, 0.82607965518188509, 0.28815416319737575], [0.15713066932678044, 0.54093839146879386, 0.63745513760888317, 0.53562895786030018, 0.64430740383211815, 0.15286762888333194, 0.03723667461329927, 0.90162644186557028, 0.66325757849073, 0.22788446447185562, 0.056273974448893038, 0.47704750973895849, 0.080940218893505778, 0.25126434204870973, 0.19370998073664791, 0.53974335953985364, 0.46942689258816095, 0.48752884465685531, 0.34096477617208421, 0.40927254387091117, 0.970756447679122, 0.25578312511509538, 0.49332202454146157, 0.26211956916091783, 0.79199773841736976, 0.7094144670119964, 0.58398713628800225, 0.33428542897272662, 0.0467374224120497, 0.83856614184501133], [0.014744462021776372, 0.56023108706724656, 0.47659267585312726, 0.73899662084823992, 0.10284855428016226, 0.97547443136160272, 0.30998152103917764, 0.6723784840374456, 0.71554214383956327, 0.14471256608577354, 0.14526599222519543, 0.080787918428295136, 0.82160180231952451, 0.46816069657117187, 0.28151705655704751, 0.51207966937590843, 0.55356922154497412, 0.46586980587080029, 0.36253155486899746, 0.36303896365060162, 0.644834999803446, 0.043105246912896855, 0.54836576700430339, 0.45364622361369922, 0.78249843461275792, 0.2994389768607667, 0.68353202464659557, 0.70848597457180174, 0.49993561085372085, 0.6600281602909065], [0.092137891217675461, 0.64027710938200755, 0.72226516782013284, 0.97962359091292317, 0.697630417397559, 0.9056963788579121, 0.97340955383028627, 0.79817118871038528, 0.13844041522017447, 0.93511256063770554, 0.55520696324585939, 0.11144059812794771, 0.43070509341155738, 0.52554407553194793, 0.71703685406104578, 0.8609350190402707, 0.51537934767855931, 0.33009818189400308, 0.948680516000054, 0.84440669507862276, 0.71942981008277407, 0.37734501702975853, 0.049644872382192839, 0.901622739944175, 0.75943770615630068, 0.11052520652694964, 0.55493656229300636, 0.0395965294284184, 0.42081983588158933, 0.94918502099866653]] [{"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}, {"LolF4A-fRknf3G-2MS-ZD4r":61948, "8iaogV-40WC1U2-CII-vdv5":12148}] +8 [[0.68849776030513066, 0.3945690554217266, 0.23162958058849181, 0.43005636234695066, 0.957092602038879, 0.91948228885547245, 0.083245187680261234, 0.80924594123290727, 0.59254527549686764, 0.31581885385248487, 0.84179038061634559, 0.48676570699450628, 0.18799174598170176, 0.270641957952527, 0.36534699824927042, 0.60822004432697108, 0.7963198854238247, 0.762516297939034, 0.9005458459567709, 0.19884478242750259, 0.31919896294241079, 0.012712557276409853, 0.73024089460035913, 0.40251978105790986, 0.16291771254423715, 0.48109078979891473, 0.8898869894955368, 0.81601245859986582, 0.68498718228715849, 0.25019864739496434], [0.47706590966593276, 0.56325831120710612, 0.8392537633568804, 0.43897273138057313, 0.85301917218994339, 0.56796091796708137, 0.34230675926282006, 0.86653241238575662, 0.34071052796556123, 0.0775203453187463, 0.63025410629159018, 0.046907238841265131, 0.86085475126953592, 0.86727564226352549, 0.85247880041860757, 0.73714060077688082, 0.31551359108809218, 0.7112736154030842, 0.15777720816833651, 0.68406360382435216, 0.42827382268542846, 0.64425237894734155, 0.5683208553966338, 0.91913527054612876, 0.18068631756127407, 0.25925536244278657, 0.80817031304045417, 0.93609610652048292, 0.94655916234729587, 0.78626718859228661], [0.5556108682202574, 0.35278401198088505, 0.91886192349020035, 0.97949015991126642, 0.72236746621866477, 0.20631298121534158, 0.44841199595301873, 0.904356155068425, 0.5277957617714889, 0.85991648821698019, 0.469178782728212, 0.56009547074301158, 0.33243540256762072, 0.6101093552752922, 0.27195424746462182, 0.89966076293354558, 0.42285082425776921, 0.47442298961060403, 0.50630370981911688, 0.21603083228172271, 0.19847308975831945, 0.312820761107925, 0.44012411210233082, 0.052395143897568164, 0.50119650232740143, 0.39000180450860733, 0.18680116291333215, 0.763447342317747, 0.14977191297792225, 0.72115313890638189], [0.82187086468070591, 0.60599165126723431, 0.0935781959392391, 0.060311139824199445, 0.054255165963644991, 0.7692933928946879, 0.9841421055158085, 0.18834590736272561, 0.39155517546397278, 0.069080912079281576, 0.3778159949220401, 0.236682237521175, 0.54454978465610093, 0.12383298972055001, 0.22931203174779091, 0.14322411701677629, 0.15296135845457082, 0.70796249545443135, 0.8133301087912812, 0.97854963648051674, 0.78271453069862207, 0.633733013645232, 0.7203064857308672, 0.80329598775419808, 0.5443275784228776, 0.95222272686000187, 0.11234847527342617, 0.921755099192636, 0.61274096850711846, 0.48337539696127818], [0.99671015744501623, 0.88205219529754963, 0.39080038943806816, 0.68615552604315289, 0.8767027982943435, 0.339881264507807, 0.051776876559952889, 0.662050906265826, 0.77372278007170359, 0.95062290273027383, 0.72071993668186973, 0.64864286537779936, 0.27911916560573691, 0.78377132982122077, 0.61010393198671953, 0.70469028857382654, 0.99091176016831517, 0.36394928187695186, 0.012141888831367686, 0.41415091265171733, 0.41790954896035248, 0.33439101018972717, 0.16830538242090443, 0.3437893755629261, 0.19256907979966753, 0.91162491708277071, 0.68600242534204281, 0.67749811976075147, 0.22397040406504853, 0.75418907030611149], [0.35303009836548194, 0.92018429752085418, 0.24357830973871497, 0.60356448441603816, 0.91233256198403123, 0.81505155230464521, 0.0489947551788793, 0.0721126097727508, 0.60275867288889284, 0.65033297719261718, 0.63469815771915339, 0.25065480826405195, 0.04592811661104812, 0.945007624693856, 0.31380375061728061, 0.38743343485339521, 0.36497865480445146, 0.097759315868661489, 0.88116872612816766, 0.75054547227932211, 0.25788133774464284, 0.9594137816310363, 0.84570429009634485, 0.92754160028737465, 0.11617439588611489, 0.44631605564678978, 0.76854688077028177, 0.23873454113186576, 0.17938077362471039, 0.25058683609112675]] [{"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}, {"u8S0HO-r2BWhhD-qsI-IR7A":32843, "DPDHVz-4OlXXwY-8iP-5Wdw":35950, "eXQQLE-dsn2YL5-pnF-PHDo":46561, "aM0fiY-csNhUC0-xUg-VdQP":42343, "SMxHk2-MgACQDJ-Kju-ZdfP":6233, "pIWKD1-BdTJveU-G3j-H9nQ":89126, "1OJCNb-ArvinwP-soL-bivi":62420, "A98HUv-XkRqxlH-bbn-Usov":15166, "rFrI2B-k7fZZjB-b2l-eutu":37959}] +9 [[0.10701536380206866, 0.25884977813910282, 0.7902525709347652, 0.4137897115249527, 0.27238753268880089, 0.54107578240098964, 0.86478988051687966, 0.20059459807190805, 0.33888160879667983, 0.719477324458027, 0.040571298808728229, 0.46250918638629723, 0.31847082537786509, 0.67561422690960515, 0.29156123431457415, 0.8061135009203203, 0.81202731508154768, 0.29049496589672441, 0.44043930814534304, 0.091635859114874707, 0.30524389711018063, 0.26375964174118882, 0.96917096610218034, 0.79149270684693851, 0.36483301998482631, 0.17143524728940474, 0.62428000079801027, 0.22852918033053127, 0.23698658464922073, 0.54867063597387045], [0.59273164948907087, 0.49257470972557993, 0.15574411204597705, 0.87858494288438926, 0.47062560728205027, 0.19427997103940375, 0.0058312768690614858, 0.24434022139938882, 0.72443571207538227, 0.85680264464862665, 0.8526944763995663, 0.05877559267722654, 0.85332354997900661, 0.66876116653541973, 0.25187133028299558, 0.724416414559621, 0.0010912250011558822, 0.52597735117348732, 0.18050123266049278, 0.29639881386868017, 0.33762814721399304, 0.774880080189408, 0.89569369127605569, 0.22182645291225611, 0.132245952458921, 0.837044008027704, 0.017636902680343569, 0.011303049383206032, 0.075083694758589781, 0.9421887196075992], [0.6597422505383056, 0.26186025329317253, 0.067806183872612058, 0.52606513438745173, 0.483785364171029, 0.7952280018763106, 0.40110544187424768, 0.46000822943058528, 0.14491778836879343, 0.80136972804500617, 0.96405630892687511, 0.1705355346742442, 0.70958759228926971, 0.78443282657054048, 0.8891644462946402, 0.880223717583038, 0.27882128221618874, 0.796739922255595, 0.82698559109443115, 0.65519601923749549, 0.22998373144259965, 0.91617436860139956, 0.86548280291018009, 0.679905740410931, 0.20204620415799757, 0.17328860982752969, 0.96182833081355057, 0.19650181314388027, 0.19255753666336217, 0.16669198759951898], [0.6381452827072065, 0.84391472586830552, 0.90892664673960155, 0.41875005105217344, 0.607723568489499, 0.71090198625224044, 0.82228952564568591, 0.11835457718754339, 0.94453373516402672, 0.27764451333277906, 0.20914710604069409, 0.2734997654762199, 0.38640082394581587, 0.56208622657691443, 0.83412508195759483, 0.45362688091160452, 0.37099056632780025, 0.21006138606938252, 0.82662045491655223, 0.24017608978937577, 0.44379777663012632, 0.97235510385372226, 0.73476373876453049, 0.22351057072481229, 0.74861739990529053, 0.99719464826205706, 0.7020400619415893, 0.88535505414627325, 0.413516480057893, 0.63327013786090147], [0.20507586122469612, 0.98811685426650731, 0.77911527084437426, 0.98623523439110417, 0.81551271566249683, 0.15593468192732474, 0.58102414145868186, 0.30073987056329154, 0.0045531431570201564, 0.17702656134244665, 0.39457254069572123, 0.35039836562301596, 0.095678093621582527, 0.18288501538362523, 0.8775047397333805, 0.40969830878612623, 0.089198890272985443, 0.26709420017818186, 0.60510684697863015, 0.054706703738078621, 0.2317215475163229, 0.29502017333629404, 0.5822678777653929, 0.99353176512337649, 0.8842646024306704, 0.62535203070835, 0.83826735779682071, 0.36285011411972279, 0.45468602591212337, 0.21544849750619832], [0.8626208665227677, 0.44341062024919875, 0.90323893896095842, 0.11467045920962138, 0.024103412960169823, 0.54527831696765894, 0.60448963020282243, 0.66335735441477361, 0.53011065306112, 0.89268205668916989, 0.1544954975678744, 0.80983113681615193, 0.92980597932038311, 0.69868287353490011, 0.049055035017865345, 0.83378852277201154, 0.64204135889630765, 0.26148002644954371, 0.027005274786166877, 0.42166879055380291, 0.49782817293994419, 0.8370242575594864, 0.27195688879107427, 0.47806056192897739, 0.82717418241774965, 0.2467088676396616, 0.596520618477706, 0.27693823648719751, 0.19139089582272206, 0.664260928077592], [0.194993537448648, 0.55749451599402144, 0.46451552866889845, 0.79255663132007292, 0.54363642859110672, 0.95156273027575222, 0.601280115497062, 0.68893607183244232, 0.5520894328004784, 0.6930988947046306, 0.094099874296593944, 0.8266944815574353, 0.12513511119234233, 0.904919829061267, 0.81784179792683631, 0.59077864089307885, 0.880934188612808, 0.88387063583137215, 0.68867338745800089, 0.29731829697397616, 0.89002145177269132, 0.71282122545701665, 0.32208088048025008, 0.99947471877584815, 0.948764817309666, 0.21636778904019915, 0.66518483283860275, 0.79618896343245427, 0.081653726577266772, 0.746842556197577], [0.39977058600451942, 0.095765579912331478, 0.678697296545625, 0.53821054403318414, 0.76498972941588639, 0.30282722537408335, 0.91737719560873132, 0.39765972817570994, 0.33247802515196512, 0.6950287514280753, 0.8930134994084159, 0.31816678247820529, 0.18167416446232287, 0.83501987602299932, 0.18706636310467029, 0.19004762286451338, 0.86941278288640877, 0.72349795778809811, 0.43981033039793549, 0.16547820285294568, 0.56540450709376033, 0.04079290810557179, 0.86198206718683978, 0.38719781870870673, 0.73400435435902767, 0.247278890572886, 0.51941645273967563, 0.73108356279217857, 0.78270932799928439, 0.017008427649886526], [0.65534986643715021, 0.93201133404428893, 0.16911495471920357, 0.46431621455427396, 0.27698379935639617, 0.96339505081079757, 0.56059946318486076, 0.889741317592146, 0.72606585215382691, 0.823022754458958, 0.77479366490706747, 0.3194687597855066, 0.97765923070432714, 0.94128422023285463, 0.39393302755571324, 0.9415917757524811, 0.96693603366764069, 0.272611963219366, 0.14247854975198526, 0.652848839866098, 0.3307433306320462, 0.75693350464964571, 0.344930965992465, 0.32109039190975641, 0.56885585015474716, 0.24515962261076196, 0.039552613566138262, 0.041581863784819761, 0.77924208344792967, 0.17120982969349319], [0.61375007440586515, 0.61935152345941169, 0.80233577006151091, 0.22280288558541639, 0.1481599985894464, 0.74514266578917809, 0.8212179688689053, 0.014216482011297327, 0.0090191007768714027, 0.84913165673566382, 0.20777933041217667, 0.29644738870783327, 0.61218889497201412, 0.4144248324434171, 0.67934649370788547, 0.5536792063321212, 0.69724468730355726, 0.611448625240822, 0.63591983788508677, 0.92969184933094962, 0.89632694925215894, 0.47469905644263666, 0.88508473422551548, 0.30382339931803759, 0.859993084036579, 0.48491823030200787, 0.82982210628004616, 0.99332150425584431, 0.71109576667607277, 0.34261696770841676]] [{"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}, {"4bsWjr-gfAfC9D-qrw-XMKR":28133}] +10 [[0.046210800798201834, 0.19270931408029479, 0.031476592245683443, 0.8179910428536965, 0.80300002985709384, 0.374844428232911, 0.11473413378304687, 0.67074038967919913, 0.70269264562020894, 0.58458553043632022, 0.24137910691272157, 0.80798996757106778, 0.85186356535586361, 0.94555284981235721, 0.36073980750727719, 0.4308928939064961, 0.98812799047625011, 0.067403918230073523, 0.8970263709298717, 0.78300851419541506, 0.5158575887984499, 0.16702206238978623, 0.1543196468389032, 0.38326004925412438, 0.77608577478954555, 0.30347925784775487, 0.93006857232465612, 0.93402232634379767, 0.537570982307226, 0.99914277143516461], [0.20150651388014917, 0.55805630917060611, 0.86493219518590192, 0.758552422421868, 0.37146718694920788, 0.8353904249423203, 0.84496662564238545, 0.64220401216524459, 0.83840108676577429, 0.83488144517409224, 0.13370402738474862, 0.10413127331560568, 0.58996411431821649, 0.96507017885459079, 0.92444832210746064, 0.76018726840977058, 0.11036963231432773, 0.0050804057947642089, 0.7310018606002725, 0.61818358834502019, 0.13876616059119673, 0.11869224160162006, 0.33046322148151264, 0.57155348370503256, 0.7192229641257013, 0.003353573155513212, 0.31643477949556809, 0.63496334031807333, 0.542476061946612, 0.580776601320225], [0.97617141099400362, 0.4833038966263149, 0.79540748321193089, 0.79689878971802652, 0.62414740660923684, 0.34475145347985314, 0.60630131982647839, 0.16637083522764351, 0.099713380508174354, 0.19409605332859137, 0.70223957450251817, 0.6703664122374059, 0.86796083284677206, 0.49429240759711845, 0.079572212662333719, 0.49639222982948739, 0.050607085083005043, 0.442196309286627, 0.32921494921522121, 0.89490857918793054, 0.75765847561603394, 0.7887078545243611, 0.75560000969203966, 0.1199545824672571, 0.29080551224525708, 0.8240499273346783, 0.58142373704700667, 0.019603956153670832, 0.14594823573695859, 0.045668654237449147], [0.33990693774281533, 0.34696112937570889, 0.2549462461736739, 0.35925021065682095, 0.022102851120887834, 0.27353919760063039, 0.57563223857396506, 0.40260621950555908, 0.66885875522661586, 0.904244915035197, 0.96432409258903462, 0.72768765116822709, 0.87535691393107962, 0.816828411875863, 0.24651521014623812, 0.12774498596939354, 0.11073164068544805, 0.77853772761431561, 0.53911820439807145, 0.35466062879596993, 0.84607869352748766, 0.828851994510113, 0.17158300463234977, 0.83573656152026565, 0.23124424268215604, 0.8595488929362235, 0.0034046750273968307, 0.769728783660708, 0.46410062875460811, 0.67199567708517127], [0.54198238617846173, 0.15867755801256112, 0.055586382307430959, 0.24643517842384333, 0.027486210858807314, 0.91509026139169947, 0.90960856414920721, 0.61917156652442551, 0.57033317164463893, 0.17329168155590091, 0.508975590374705, 0.33094312180906271, 0.50206991675733426, 0.3292641100386936, 0.52684892313456977, 0.88340207422186434, 0.33190439599205968, 0.13505771934106636, 0.86777273960568491, 0.020504849796612823, 0.088824292013345518, 0.38185921507766307, 0.48316047736858925, 0.93774801067911218, 0.37318524186720559, 0.32006473685349024, 0.303187916625283, 0.71456893243575, 0.14184714869290505, 0.985210849808247], [0.28036230596193623, 0.36386234856278232, 0.10738247704220172, 0.025375177204636667, 0.84960801013553766, 0.89744716702706406, 0.64457524424388968, 0.458666011573034, 0.075528707542807827, 0.717450505168633, 0.26637523812493846, 0.028678003449522316, 0.37957704832857153, 0.46095882986695513, 0.056667034878263522, 0.47129925439590359, 0.018182770643203727, 0.14891883346932688, 0.92865861362550228, 0.058012255245735944, 0.41746062034714981, 0.12015358022974976, 0.21539000467968494, 0.23883962967601102, 0.06093955164774556, 0.76749218018286058, 0.89280734908028558, 0.072342885050724171, 0.87382561601856223, 0.53539411530574021], [0.68604341816330061, 0.40059051258986023, 0.95063020070153537, 0.98708204357627138, 0.41599065793372536, 0.40376969969812004, 0.032566700626776912, 0.0040799765619552453, 0.205948167303738, 0.25212317170720766, 0.35693833499536032, 0.34218631610155836, 0.58706972164936111, 0.06215111608768864, 0.37281030446317376, 0.83529382231078475, 0.1165345980110557, 0.25952570301105116, 0.34557987316486805, 0.51210225633713446, 0.85380887020074392, 0.75292341526863349, 0.057628087813190509, 0.77065111246784956, 0.92805984550963072, 0.030988327388477233, 0.65305753336446981, 0.54173348906723, 0.47373170603891934, 0.40889734521187138], [0.17159035564538183, 0.064634855560365634, 0.41148572006193351, 0.17150621829720369, 0.47002136521479254, 0.49727261091734076, 0.90815369167824, 0.28681273419927711, 0.039191390110807744, 0.087696401220295983, 0.86204863523926911, 0.69648150687510446, 0.69026631701875818, 0.045040431346139775, 0.42892208401761955, 0.92943286826795646, 0.20617734648260511, 0.56540347373812549, 0.429052463719241, 0.93000526611910317, 0.83239857352356028, 0.69176331465323482, 0.45739469828608048, 0.5439596299136783, 0.91580841987333483, 0.816917033408578, 0.65641599907653148, 0.063600950037368209, 0.091201226039362981, 0.6745696900560948], [0.40518874980815778, 0.59755059269797783, 0.16208623383376741, 0.6060778706714689, 0.62850596070081954, 0.801210677997065, 0.2257311666541838, 0.73898705393449726, 0.0362308764983994, 0.68071953834963839, 0.65197786759805276, 0.46838425900869018, 0.36578684032881881, 0.76620516216876189, 0.24117813343423367, 0.2932207622638171, 0.61963964243459013, 0.93940137493822962, 0.25866767424461445, 0.627142160196391, 0.36481901161199692, 0.59619909297283968, 0.0715170756428598, 0.67699578087331547, 0.3929720501020344, 0.6264631089749213, 0.797868179877844, 0.86206882358560966, 0.42713618514011964, 0.90145268357145447]] [{"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}, {"zgPowY-f3jxL8U-cS4-8QSI":29935, "j4nuRk-A2rCKmc-TaN-Wi0H":83902, "wmOnOw-ekBJ24I-uwh-1ilG":66404, "bRkniO-MQ1Wy1f-tJC-Lk0z":17106, "rjOwI5-S5HQEqc-arA-LBc4":39625}] +11 [[0.65327432894692838, 0.32894198092623417, 0.9619307687845664, 0.99539125926505623, 0.53345764975478194, 0.86505853674749433, 0.18498367713182573, 0.11584283599472811, 0.19985143801523308, 0.022159199001309582, 0.606595360826325, 0.64825041300866781, 0.61877978755153007, 0.2018765340783063, 0.69951173647581022, 0.61528678466458853, 0.55314671839668983, 0.0642892129530197, 0.8654015226776085, 0.4606225640448115, 0.11353434062603562, 0.642277842163975, 0.13033174785619572, 0.9286040206797721, 0.03736384810835025, 0.74278867889748146, 0.45046214004600893, 0.17757070873604874, 0.61790238835819478, 0.18789277792224235], [0.07171479611701026, 0.7368287581787617, 0.97540744926463485, 0.85184717090624229, 0.23580777421094024, 0.52374078447814354, 0.53945748233680357, 0.50993633215001266, 0.27992450944595393, 0.90008926061840022, 0.194525950970871, 0.23069140239883068, 0.838423356611276, 0.66581465330948608, 0.5301133477109865, 0.41515538785953643, 0.422375081564913, 0.9634261700731509, 0.032259665280996885, 0.61518320061890575, 0.12634771552944124, 0.29782664808806236, 0.855017676653127, 0.94733490135742648, 0.591742726133152, 0.26399030213079355, 0.018475283902462669, 0.25974184070831441, 0.64211732975082525, 0.11887537646636637], [0.49417264233369018, 0.20639450380965241, 0.19831216872095392, 0.034151037233477677, 0.65926805176013281, 0.036622556612710966, 0.17983563329686836, 0.72276149814330637, 0.02756401723115609, 0.21710658242958947, 0.10383895097174212, 0.14504663675343465, 0.9985128921574451, 0.96958220041149823, 0.0041490798461749634, 0.16908340159150115, 0.60457953162803391, 0.42946735118633017, 0.11020947268541958, 0.38510765190517016, 0.88284556828327831, 0.40896336811684508, 0.44941108977633015, 0.7553212498698556, 0.5484983933690798, 0.68275185841162878, 0.16617936017644275, 0.41808477613518424, 0.10585553578239282, 0.30522817960137538]] [{"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}, {"jhu5jm-RKu0kVg-APr-JZu8":61583, "zjHqDX-ACWXdTq-qPM-mjti":19506}] +12 [[0.81185833049958844, 0.30666067502430072, 0.65538831497946615, 0.203106185079865, 0.00078869097953226941, 0.0031811868122340048, 0.265963761865172, 0.35560473239960733, 0.86359735984662778, 0.947279996928223, 0.21334787657408905, 0.86788450171151021, 0.22469454256582055, 0.60451736003698042, 0.97510032079053133, 0.39843263106573423, 0.94130813218741072, 0.0459891766655921, 0.1620075117346309, 0.64544667605197537, 0.67346080605993008, 0.12485844879470631, 0.66225160763659563, 0.7444093017507516, 0.10802675728658417, 0.77509559466729094, 0.47148574969643342, 0.606246096671366, 0.837093877912571, 0.034264285884160861], [0.10484242091193363, 0.10698604401641931, 0.49027143778178461, 0.83066693080528431, 0.026940343446078874, 0.24394496948341471, 0.65124682820104507, 0.794889767327406, 0.10724509563993656, 0.85880456857967635, 0.55080974485250633, 0.35676466588939149, 0.71067665182969331, 0.29456071787847493, 0.52172591541682145, 0.17099091509039177, 0.66714720709875552, 0.40116670543867361, 0.7252560430818068, 0.47813694280740593, 0.02875095147654827, 0.63379809191616776, 0.71478090015890627, 0.35246866813325151, 0.2196167947100689, 0.59801811756733336, 0.98890605566899437, 0.914027238765278, 0.91097426619299116, 0.28772354699913649], [0.80458524923354779, 0.4896063708219377, 0.7127715375099607, 0.43460258991311562, 0.17021432808035486, 0.39575488657685065, 0.56322366121268741, 0.84193842637233507, 0.11931250420003925, 0.48665763181268684, 0.51895851007557448, 0.31094285621035433, 0.077321141711810815, 0.86072402311973328, 0.500966597408782, 0.7844066212502121, 0.37721509212767568, 0.15194017226364476, 0.43535003391940696, 0.63030436198049256, 0.75403477721160017, 0.64551681664229055, 0.14290519599641871, 0.56709348878537646, 0.95411193456001286, 0.052092567453138727, 0.27786258850103562, 0.59279832111234865, 0.43167589510370286, 0.15072748398310443]] [{"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}, {"SbvdSy-QayG0qx-j1K-oA8W":92219, "kDmXep-gMtx9m2-7wp-5glC":38896, "RRQrvV-g9HlTo5-HqA-wTg7":34643, "JLVlDd-Y4Ngn2z-a1S-bXlu":92092, "qBKk93-20OaQ32-soq-f6At":78734, "80misd-kFANdp9-Th6-ULI3":5590, "hwXFxh-31LHRpI-85G-7QMZ":8996, "Lf0UHb-ChMEv84-PhO-cGtv":39791, "TW1Mj8-WfbHQsP-5PT-pRDv":70995, "cbQOb1-eli2rlF-d7Y-vRDP":85847}] +13 [[0.12030761117169952, 0.21550887718555545, 0.49410894070815692, 0.42143467974520266, 0.81946279547105316, 0.45846426055389233, 0.65712742631890353, 0.15314477366832269, 0.41528878608123254, 0.12911772418269263, 0.67926389722739877, 0.93346414159677027, 0.93520902500503, 0.15421306709632598, 0.4625297969625134, 0.59796307240740187, 0.64657805418685965, 0.49975378919957991, 0.95603621000037775, 0.59442924775666361, 0.014895576314810866, 0.24932905647433645, 0.8320104371213185, 0.004921698921256823, 0.41486368996123879, 0.00877637834400713, 0.84954657070891948, 0.37800239275831116, 0.96528894570896462, 0.35803698662685224], [0.98656421020514684, 0.83663040078754358, 0.0043397755645839808, 0.84937980578720562, 0.64408353518608, 0.90864826936267229, 0.27342668654434943, 0.48574018492679705, 0.35680704565933652, 0.66448618463199893, 0.8359810167149393, 0.70374055096234978, 0.4801311131192485, 0.040653982267929045, 0.12166583771045647, 0.63302920220490511, 0.26609373380369927, 0.67778757995213024, 0.25706004985142905, 0.80689531575898643, 0.5893971218669638, 0.20846022706427592, 0.020614688926453684, 0.5586870942440505, 0.33952699176916423, 0.69266579298379982, 0.342688813351968, 0.054381340736341155, 0.15612317027708744, 0.81782523711478494], [0.26072892103325451, 0.28372249791459037, 0.75213869808471556, 0.55536400811949049, 0.63516508245504011, 0.36066994077238823, 0.54295766365991416, 0.19735228089662404, 0.73335375923466883, 0.16846182185812264, 0.67952492734942482, 0.084396098112331841, 0.99690539046973914, 0.72542520776749964, 0.869673045981151, 0.87991236032954057, 0.9219930806124127, 0.64473169653841311, 0.56421571979509466, 0.94611596332177261, 0.0027455923858757947, 0.714618162816727, 0.18300846926266867, 0.23349301928575728, 0.11458677859290856, 0.93516310042444328, 0.061061120316768247, 0.10822876868555731, 0.5473903903258891, 0.3135285066352077], [0.70591929252047192, 0.71100484575296885, 0.13757775037374376, 0.3416483145749144, 0.77491429511675114, 0.76577532102563506, 0.5937931142032723, 0.35866452632223034, 0.90451454807372766, 0.23857652469052182, 0.16254969822901377, 0.93498183181651284, 0.15389362561784659, 0.88412084792207946, 0.23824681625892807, 0.0056330166758079558, 0.36096357638862353, 0.51187010700685165, 0.3862168499310944, 0.72771673786372459, 0.15150431211997206, 0.75347941331206414, 0.51796803989538986, 0.93998990080316447, 0.69065321316005146, 0.2199474932592913, 0.63454721389116353, 0.662267623517278, 0.52090143900991992, 0.19950529810320083], [0.6369545971518068, 0.28810314746866161, 0.99187675214663484, 0.46621934301890944, 0.19180007937524712, 0.99629134412332487, 0.85457505635017283, 0.34448232018796265, 0.53589851407395017, 0.43505915421231134, 0.56633576198828772, 0.90600787788807025, 0.40525963816842869, 0.91737679718914966, 0.732739147700151, 0.063642831930431187, 0.98193529205762775, 0.20664390310775638, 0.89873627109446652, 0.370254617750018, 0.86644537098877217, 0.46907513119506405, 0.59370529976612019, 0.53114716834641618, 0.60401092009124335, 0.12242991663183334, 0.13376040628288555, 0.994904466084115, 0.2195809930672119, 0.12030147454189011], [0.1895226233581746, 0.50025502327786375, 0.54964791263575818, 0.6770005514826, 0.5386947235853734, 0.12549726588242349, 0.89810274375598942, 0.56013947101381911, 0.74016483810807032, 0.21221998126801911, 0.68889132378005458, 0.85634269507084348, 0.68999657767096323, 0.51649804083558193, 0.33998852400924851, 0.55008001616985336, 0.14954616728387149, 0.078438466371422155, 0.54121630298259982, 0.34643920135627515, 0.37207971614882851, 0.59489746800676224, 0.48268921049075175, 0.91572511596516992, 0.89664700730924707, 0.53038437285810514, 0.35117835910424855, 0.14109222761097551, 0.42660037814593355, 0.807613476286692], [0.58844312116110731, 0.00998861219290037, 0.4650307277507405, 0.68186046768271635, 0.98858741375534365, 0.50600886093536179, 0.040525031267945155, 0.32222525607820651, 0.70075511756166153, 0.22982447998082944, 0.42403275264536289, 0.69809801594245557, 0.59898726514736889, 0.50618006944936922, 0.21126128710859504, 0.60057132701195648, 0.28282143040615515, 0.41851428547131242, 0.52490343539402429, 0.016235801150929596, 0.18022113369038162, 0.55358455324863309, 0.3771328537445594, 0.61474240524935209, 0.83868159683975252, 0.23554515711942758, 0.42518564712515672, 0.81170982018846149, 0.51086526070376681, 0.68224837358617241], [0.45148014733742969, 0.72500056071217545, 0.56787879714221456, 0.826108957217412, 0.90460847899533336, 0.35508477260189741, 0.10631113462399189, 0.50769188321529624, 0.748667488335067, 0.48915600929197278, 0.35440940554012168, 0.37225471488555029, 0.68577843281665418, 0.15385878057663571, 0.12827807020774495, 0.07779506732223207, 0.54027842243752866, 0.92293253118975671, 0.64074379541355664, 0.35772483577421488, 0.22294904599228582, 0.023772496506828666, 0.42353712489606921, 0.07211266152200646, 0.22978018867740468, 0.98085300200016134, 0.98769883358254307, 0.62850673993529271, 0.072150585920248589, 0.48415383332080109]] [{"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}, {"9ErCXN-g6RUw29-wsE-xsTH":47137, "hN6Syb-Z8po7gD-5fL-047g":39092, "XSxMze-yvK1z8o-GpM-EXEJ":44715, "rzPGQD-dTUncaW-5bm-xCQw":39027, "ITa5Uh-kzc0I6Y-yjw-UMt8":40225, "TVstS7-uMyieUC-Hrk-kt7o":7617}] +14 [[0.56756937645255057, 0.25931061792936327, 0.42232614384641554, 0.85459627004177552, 0.45020684777715936, 0.29767402545492538, 0.080229346108791177, 0.022898418006150334, 0.080332362797224555, 0.32680510799539164, 0.3650687261202672, 0.73586028167479645, 0.8973098989709124, 0.2578144961028338, 0.48980318727312011, 0.58474995051373668, 0.463153911850928, 0.011714371147868552, 0.43605207683782576, 0.63128663585656686, 0.066994296286058352, 0.60624375792077911, 0.030652471942043236, 0.62005011073507343, 0.48332817779826132, 0.084140408547301315, 0.47814224854841747, 0.23352396165508849, 0.34990896299393481, 0.77532205953952349], [0.76082076703846624, 0.441317043535284, 0.53053807222898774, 0.32391510683087321, 0.35807783006852167, 0.69920902070243862, 0.2586876845487861, 0.63079654219839232, 0.807970470981805, 0.73901699086123318, 0.8197980996131321, 0.36329070919547579, 0.736566665581185, 0.95688238564364836, 0.023256404137849462, 0.96588930115593807, 0.11436357246112405, 0.95542837061721053, 0.73947771682625274, 0.16933972668348973, 0.66412733451568728, 0.624340005062185, 0.33317430456438091, 0.84017318148392273, 0.54079035636850725, 0.25427021734002542, 0.91081164163890538, 0.63872062191874324, 0.63158133749035772, 0.57057742391975075], [0.91841487227229, 0.36089141138668757, 0.0833518326748457, 0.24625083915245694, 0.77530312157709935, 0.027465482056069579, 0.95410265258246, 0.40748218057585917, 0.90999291952892269, 0.75437735872720779, 0.20700262142983805, 0.88069329425735221, 0.776426506111379, 0.18844579712505505, 0.61773040604236962, 0.44643123138128971, 0.17184430446704302, 0.825230491550162, 0.5497603304387092, 0.16339366811118872, 0.45203552846210016, 0.83917282130172033, 0.96725134406770408, 0.904949199582363, 0.045364257947330566, 0.15090510395479428, 0.50148752549809072, 0.61086852357139487, 0.0038831854931207221, 0.33258867054162888], [0.864358110524949, 0.39858412816678412, 0.19801049814889682, 0.60936788065731673, 0.4111529183394369, 0.601147840432371, 0.16065988275179388, 0.65350426902971726, 0.19498974156276028, 0.0031566451467059942, 0.51737780925894017, 0.087480377307895263, 0.42647762629326036, 0.03126954097302348, 0.40095578528154796, 0.61334072256343375, 0.8547604341585473, 0.42253887073246943, 0.24214990009059389, 0.46776306359667008, 0.21987309156965695, 0.94163191048017258, 0.85255375069229955, 0.46582988114719315, 0.5913829324872949, 0.047119474081575685, 0.77133486882279689, 0.24119681994933717, 0.10457659612479175, 0.017360270140969303], [0.4344590756876251, 0.41633091478389228, 0.33299295423267627, 0.9381109781083472, 0.843190872002207, 0.34371843798919854, 0.29489092275591211, 0.12684395049835939, 0.058072439984518609, 0.747418627488594, 0.881280516441675, 0.022855548675478565, 0.025247720503955517, 0.0640659485109788, 0.31443421015326867, 0.30760003462143948, 0.95391189884505145, 0.53340023665257275, 0.098334918318898179, 0.036285552917449726, 0.70683106764649006, 0.12063067732060517, 0.80630626197886757, 0.85961986703067839, 0.44029065833304959, 0.16641629062876828, 0.96610900959379764, 0.993695368886162, 0.03549523253226905, 0.25497148760779931]] [{"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}, {"IhRK2i-DZkc06u-if1-4VAF":79543, "NWg1hA-U6lndND-0NU-Eg4Y":28408, "7Hoq7k-ZVzJxfq-bNV-C2Ky":11791, "ixng9k-zH7APbh-AKk-u34O":10208, "r1at9S-mO0rHbg-uJv-6tSq":61924, "L3HlGc-CsNTubQ-3Wn-ErYc":14987, "OanPRp-Gnh0XID-SR8-0Jsf":58895, "IruQpg-6ReZdDG-lUX-kHTh":2373}] +15 [[0.65069461186175015, 0.72665034408710583, 0.77197434126543019, 0.40282943352113187, 0.64552393492322058, 0.765940755594443, 0.22553712288078731, 0.27967614535782626, 0.95223357133615949, 0.54526938922304757, 0.77496235494751209, 0.96470331564816869, 0.23770553340681233, 0.667868020195893, 0.71343109818229344, 0.59255155272705151, 0.15507542962345877, 0.35088308415654912, 0.46178941443891897, 0.24185122040952867, 0.35371780010244869, 0.47385360009386546, 0.69938190468420414, 0.044096788043858748, 0.567208730056698, 0.43637481904316988, 0.97552660164627736, 0.20175929227241229, 0.82018956127708165, 0.88760072011744939], [0.77077791579441168, 0.69166791769806091, 0.878063908524915, 0.60605770050101193, 0.13825734047055815, 0.84863795738609138, 0.552717342036034, 0.68031316989944823, 0.12271404474879566, 0.59951680568463339, 0.37295750752780021, 0.66936220946748792, 0.12861061735106916, 0.84926592031265957, 0.43012996051128904, 0.60616635809298935, 0.2884119839110979, 0.40839108314888073, 0.91737268012673845, 0.54859853111271917, 0.66838827558220493, 0.894979202023229, 0.9952304120654264, 0.87911869720148217, 0.84162510492326315, 0.24479501938510084, 0.13685750812493047, 0.707824882304903, 0.95656796082104734, 0.8193363965746171], [0.032795839549656147, 0.54309470598454945, 0.93430959898698562, 0.61444210103741137, 0.46832218569743367, 0.85985292742969355, 0.83896276248739732, 0.7409915362231202, 0.32561723874588178, 0.31757063898256332, 0.78284291874184575, 0.17443087503474852, 0.0085163469346685172, 0.1584218688292548, 0.95181871869239532, 0.75130563063171019, 0.13935976208647116, 0.75059457401479368, 0.46607078064580942, 0.528675033372055, 0.33546790889519307, 0.33257198228806806, 0.82763174348737478, 0.91622829227898994, 0.23702535173805139, 0.76127159583044646, 0.64846421400657173, 0.44111934742373327, 0.72866450746561673, 0.47024925088637426], [0.20146492800226823, 0.61198055097914938, 0.47379836397733044, 0.90374550757870242, 0.0916674759852596, 0.11127234794499319, 0.46375887527171478, 0.40817235193607437, 0.0060475880006942706, 0.46407032177532104, 0.27305596809932686, 0.934708669401926, 0.06771860929361917, 0.40658715728149475, 0.10757942520312835, 0.93499270409623181, 0.79393734469717359, 0.40917150245570577, 0.53432424255632982, 0.51647056880371278, 0.037223245839366093, 0.19622183932949233, 0.98922726213584167, 0.26477968619976855, 0.5750264400002475, 0.64939818730661147, 0.77396135668720067, 0.13761698286626478, 0.20576370871230509, 0.18071017457798044], [0.27304077465611365, 0.48786136981005057, 0.30767343146591519, 0.51850505530816027, 0.54934909601146364, 0.5832905828379199, 0.82182387420486624, 0.86207462065582108, 0.30920936464618076, 0.95605288448953252, 0.53174183320361468, 0.52954544173854945, 0.46737373946818739, 0.537124416482234, 0.32951155568702251, 0.5489121049450294, 0.18781708655579721, 0.8093028993601924, 0.88182652764419578, 0.12351799493042914, 0.32797809046648918, 0.14685949868905013, 0.73935953413090649, 0.54098447673842376, 0.67209791068212466, 0.10078998976063447, 0.46894956656617448, 0.88853341969470123, 0.20356518583992211, 0.63567767006747222], [0.8304862892999354, 0.8878444219650119, 0.93668353874422894, 0.22749988666965393, 0.36900637255423385, 0.26903254443142943, 0.50503555162680547, 0.29128108068104674, 0.047398055586663435, 0.366028377388672, 0.78662529981017837, 0.623314997721532, 0.70624375812058249, 0.034586878167038204, 0.3352389395506904, 0.41474949589797394, 0.20471304044733141, 0.56934753787614578, 0.28297101761010224, 0.92164733904388418, 0.070050855619523622, 0.027812872185748683, 0.68062177982366889, 0.061206729362587975, 0.36308943686791417, 0.25680830737538873, 0.95067848905191088, 0.044646154984623432, 0.25334429322945795, 0.96093131326474956], [0.24185188764374865, 0.65432728830221787, 0.48963811996369944, 0.35269327974971165, 0.99464648732477734, 0.92601156612453539, 0.55082892619951285, 0.28050283221305439, 0.95420121942474534, 0.80671473734134469, 0.7661016516966197, 0.24333431617559642, 0.1996448325495358, 0.82444538108099752, 0.10541527237050397, 0.36987117079037768, 0.49846835769094688, 0.92818278408078059, 0.658080431658304, 0.26510851467697838, 0.43992451041632041, 0.43477033054295244, 0.041074894112197224, 0.60884495973798547, 0.35322181124764407, 0.4539705851632051, 0.10858769985945937, 0.52818883966955277, 0.9742893147133761, 0.25374904530718068], [0.37041504704993122, 0.060149643547271592, 0.18914514645810887, 0.77492316432407427, 0.46921414751716228, 0.4705801400326729, 0.95241010594607223, 0.90763407400888851, 0.91525386007431, 0.80218069957878435, 0.65626568443862732, 0.20166865721607652, 0.03218389393473331, 0.78697222672484979, 0.80816300548785924, 0.39847986743588282, 0.72847716584134892, 0.17758934757320333, 0.86484765334229752, 0.54482325215440741, 0.27658260781751787, 0.626941148242353, 0.4657728037267147, 0.47624856901173263, 0.755458447422461, 0.47723196422221514, 0.11073101645313965, 0.14325514669026695, 0.19203938955821453, 0.20702959732944526], [0.5345427302854624, 0.59374014345641823, 0.723322848948313, 0.82901458510530113, 0.645897558165222, 0.42243909696500903, 0.29314345360539429, 0.23123825034142098, 0.91795196163706194, 0.1513787651662527, 0.49633682766464482, 0.31116374591329, 0.54504511329222427, 0.31493215704602551, 0.18787698157969335, 0.76445883110795176, 0.51290203654499311, 0.98888619948725409, 0.38570447720047185, 0.5594546354827572, 0.96895146962807954, 0.12500271418802067, 0.81334516289958048, 0.45338036542178273, 0.92489338328647719, 0.33087382275227117, 0.15205186849876295, 0.38792828194869866, 0.37540892606166032, 0.66944315645043551], [0.70208514375847, 0.23939990228801911, 0.37491340476014368, 0.64338417071823273, 0.68888056486988436, 0.19262845396691919, 0.47851099891324989, 0.77659816685596861, 0.89223455103779326, 0.95806868377131016, 0.22381709446788556, 0.10841353461395231, 0.62527170977196622, 0.15789015477575596, 0.0070725046649237067, 0.47563952441333834, 0.34484131944837926, 0.46960546425926364, 0.7899816518242524, 0.6734468595133104, 0.33612179162441869, 0.53471039849915591, 0.56296396786019709, 0.45811874308755784, 0.9815958237608855, 0.21500777471488219, 0.3860163603557687, 0.883035830944893, 0.098191402302366892, 0.51073285588808714]] [{"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}, {"xLDMcY-0wHo8K8-MH1-bJAs":99471, "qa1gmn-spJenBY-dZn-7eqQ":36149, "CLjw96-cZQQEKs-n3l-kIs1":73424, "7Xh2jY-EIDPRVr-bfv-QQTS":30561, "9RRpFw-toocfRj-lQn-AKPM":47831, "70Cizc-Ai8KaJV-tO7-grN7":70696}] +16 [[0.35441253712880016, 0.74230695184923845, 0.403537363109042, 0.41356457856567996, 0.069869469799674011, 0.26499751959177964, 0.26997623778465574, 0.68996237650176429, 0.68746833941028807, 0.91859472331559722, 0.92932073879157817, 0.68144804533134318, 0.30166772591661484, 0.3655925371376133, 0.33436780641007413, 0.73591415210585975, 0.40855237634638375, 0.70179921551863145, 0.67963263037462351, 0.5740227195464046, 0.23769397114716284, 0.55289746325245326, 0.92133186037084069, 0.73162152743413067, 0.97887992335314644, 0.993949924387691, 0.44737681664678652, 0.7791006656499333, 0.55486346057371194, 0.61609979618904043], [0.021674784824366577, 0.572114466357809, 0.98289037130124668, 0.68287939187360713, 0.53426339745090856, 0.046883000975285416, 0.040609178349845787, 0.46795450374648995, 0.27727884842198269, 0.36332622786411506, 0.90607249099812626, 0.54679565671210384, 0.43648620556398654, 0.52057300653524108, 0.67689010389158577, 0.026681964901016486, 0.23907579020911773, 0.92192647908806868, 0.19019452500968326, 0.44260716555915047, 0.7306567140780752, 0.75015580592098918, 0.36293230203116589, 0.32107142826908264, 0.46785507234112034, 0.42902406132396775, 0.35109546980254958, 0.21940647069208719, 0.50765058340243441, 0.89325063769159574], [0.18685925413111537, 0.38519633781118423, 0.95056266892615315, 0.86664551821219782, 0.19419351258197026, 0.048518746172414806, 0.95202280201738165, 0.85694240600697591, 0.40636747835268616, 0.75914779050281145, 0.55474692068297082, 0.11589596095833299, 0.41286794855345188, 0.11152145611466002, 0.91464830270759268, 0.38449827411068505, 0.54860395293395525, 0.6621920665988672, 0.49380761629765035, 0.8457080892075497, 0.10487107197440837, 0.8376417359484315, 0.57624364332686062, 0.34087578424987164, 0.28940139812221966, 0.15475905601162743, 0.02570455449560527, 0.57108597968421271, 0.99192588243623447, 0.13193711251291951], [0.54942637305173792, 0.70092878272876979, 0.022027625142207574, 0.85566720105355765, 0.85778166575656656, 0.509749721989981, 0.37766630090871178, 0.28566639954243811, 0.016968187266130141, 0.99421738781391378, 0.27560543252781966, 0.030991682207522686, 0.98036343197659737, 0.11608717839416238, 0.12907568483311171, 0.5102566998560657, 0.753898380275559, 0.21032807802177689, 0.39335908683556942, 0.12098195146344415, 0.90177919871708945, 0.42131910347185719, 0.21658038894996634, 0.2862728963245682, 0.13111336371703564, 0.41861042907965884, 0.13084375630872791, 0.021434617177623627, 0.50747214069654789, 0.55878395257768632], [0.59795407919148358, 0.968795734291013, 0.44814897593009861, 0.60623651003530055, 0.0843140462690497, 0.68592840339197536, 0.0158489296505282, 0.10324848835077227, 0.48118958357748132, 0.3406150339804499, 0.19277164072760289, 0.48848929544269626, 0.10938796755218372, 0.86532697150255, 0.7535263511254725, 0.10315050678818716, 0.25149411410468647, 0.054643785397782252, 0.075044437730137581, 0.2951863499687396, 0.14016265812699924, 0.050956245258205968, 0.72263109069516218, 0.93468017206214837, 0.59756436789679257, 0.15696010383345582, 0.93352476110085225, 0.78714510509527358, 0.068937779432996837, 0.93786204933055251], [0.1828091522323706, 0.35660557381301861, 0.622297542770797, 0.13188243339764161, 0.86124944258005232, 0.0065956638862136874, 0.59670342166609769, 0.97455485889821247, 0.72060079344830619, 0.39474952508461258, 0.97309485744339785, 0.1604850884612492, 0.149163680380216, 0.51544781857276334, 0.21454802976504617, 0.81564741833246523, 0.52883758289346261, 0.93502334129373843, 0.27691333586149114, 0.533959587479671, 0.89465025333782389, 0.12531826908045518, 0.72614334079091214, 0.743791706052353, 0.8687176796993008, 0.15392398510112415, 0.4152225576392552, 0.41322892123165933, 0.37281483289142914, 0.373224257864461]] [{"3tcH9b-tLgl3YF-UYc-oWZF":96074, "33rPqs-QwIgcnd-UXp-KjKI":85207, "R7In63-s3eWDRp-9Qt-SMhD":37556, "B5LE28-ZmkoTSE-A5m-WMrJ":59361, "Ixb157-ZePShy7-H9q-G3s7":27515, "aIIm1A-iXCXCbU-8nO-kzBy":49574, "oIDF5o-lHZOz4k-A7S-G3fp":10202, "EqUdAF-LfpxS5h-tBM-w9cR":17228, "wLoGIQ-tkE6y7d-kZY-Z9Ao":20205}, {"3tcH9b-tLgl3YF-UYc-oWZF":96074, "33rPqs-QwIgcnd-UXp-KjKI":85207, "R7In63-s3eWDRp-9Qt-SMhD":37556, "B5LE28-ZmkoTSE-A5m-WMrJ":59361, "Ixb157-ZePShy7-H9q-G3s7":27515, "aIIm1A-iXCXCbU-8nO-kzBy":49574, "oIDF5o-lHZOz4k-A7S-G3fp":10202, "EqUdAF-LfpxS5h-tBM-w9cR":17228, "wLoGIQ-tkE6y7d-kZY-Z9Ao":20205}] +17 [[0.97116668791348648, 0.47225829864239022, 0.48518800874288348, 0.12778390066476486, 0.66738280266453376, 0.69756208231553318, 0.58691193681800757, 0.83726240370669713, 0.6342386538366106, 0.74162195510476436, 0.18754811726602427, 0.72903016794612419, 0.029662560754477374, 0.61507586494668487, 0.81733460175545913, 0.8932036744394557, 0.46655828523063747, 0.82072526286742431, 0.77657049415927248, 0.833949582656972, 0.12484821467354346, 0.88577116031139791, 0.982082275780593, 0.23944382406497233, 0.12939781292324037, 0.46834988139007749, 0.32268977838427482, 0.2363626321919553, 0.20666984843600844, 0.27802324450910476], [0.20302386922242821, 0.60147096817582912, 0.0050258369610359788, 0.61726051685264161, 0.030242892390883958, 0.79838032598306841, 0.49191411112674843, 0.82458213566827931, 0.62964026399704942, 0.50930647233469706, 0.52575255978398172, 0.27991823983197217, 0.96283089772297081, 0.53966525303108481, 0.041135728333564159, 0.91886804184433779, 0.06614990867474857, 0.58760231232605664, 0.28306543709178233, 0.39492823425804258, 0.82709132506633376, 0.391060440161891, 0.17922493429735875, 0.12739221563621561, 0.073061039681284634, 0.19174649205667915, 0.24888638004519059, 0.47750400396515058, 0.45941269445820221, 0.2883667791065001], [0.412862362617056, 0.20658440923480914, 0.812681024320184, 0.69075828744408785, 0.69098186968779607, 0.18722866616986078, 0.910121946725318, 0.53205101612402728, 0.051892087705240231, 0.062308663783570806, 0.1581147555397675, 0.94775392606693765, 0.61455697567015721, 0.17639511232209948, 0.26967631560557581, 0.58866504991466184, 0.99164457089824209, 0.64034790405812858, 0.83528295921027806, 0.79876474658801422, 0.81206485622870717, 0.83220579232098246, 0.63139096511723591, 0.75975770452747227, 0.36324044928581278, 0.98613934801669934, 0.61749175523892541, 0.58068827636043818, 0.853262783655353, 0.77611382058540312], [0.83523333303482183, 0.39406071393729658, 0.0089066454926266525, 0.96204153750429555, 0.84208478777363882, 0.0091541147655833655, 0.59754608594568348, 0.23358624192734911, 0.88721581050952136, 0.56872970501594677, 0.98933581843314311, 0.36041433656082, 0.76870907955883094, 0.95708550027754857, 0.060591026509963841, 0.63757116316855633, 0.19891764653914001, 0.41542171440630193, 0.58012062582148582, 0.40079318790378848, 0.548725962946865, 0.93473789000914764, 0.71733802008541325, 0.078949461788597963, 0.041667580665553205, 0.51699022719019438, 0.46429387061193617, 0.84943892188170933, 0.825298235177949, 0.3219960034289634]] [{"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}, {"qbsozl-SKpguWm-HbE-QllF":45080, "5S1lMT-FZd2bp5-AVB-ES8a":68487, "FD3QDZ-QyhWRW1-SNn-PQG7":86117, "4dI9RP-EYIRfOw-LHT-yKvc":98835}] +18 [[0.3363835250565157, 0.42751606296856792, 0.95909113566627757, 0.015305717549693898, 0.63963346692763268, 0.662206475764985, 0.19265081668227968, 0.36648829855161447, 0.67768234725844145, 0.44679057950003875, 0.94253566163541913, 0.0419972030351593, 0.88215408366970394, 0.63119869767184589, 0.19829737844392692, 0.94022059322342155, 0.82699301653092216, 0.67549611064588866, 0.20717354676738475, 0.73738381032610623, 0.739135700713662, 0.5276357210213487, 0.12251055206492134, 0.87199014141755016, 0.47051403840344674, 0.94930513747355894, 0.82367703710708573, 0.55189507748263777, 0.78847589969397591, 0.42454220893889616], [0.8138707834129465, 0.34691087630813566, 0.10440491569846055, 0.86880075736393825, 0.31804157739550953, 0.090970893430076027, 0.17926605644194693, 0.84540648442663369, 0.57522357752851816, 0.88526616315031581, 0.91999911493853148, 0.89185035004328017, 0.8478905507068244, 0.96739925034111662, 0.11212771356189832, 0.86012482193249007, 0.85551778689317015, 0.091483510347962649, 0.82164633304019841, 0.8833160392196544, 0.18035492017878907, 0.85024835198148263, 0.24319365806536042, 0.69492302800354078, 0.69048416232444165, 0.062594312969588173, 0.97171543356199541, 0.54788250558115881, 0.17855857679116593, 0.94867419350117033], [0.031265477524745844, 0.90703607725982471, 0.22418561306493434, 0.31945959704555194, 0.50728920198202476, 0.42178393253363022, 0.7537411843487577, 0.4188848361868045, 0.29767846296246736, 0.42492751242319071, 0.89347014952084336, 0.3740431241351827, 0.32308253514173324, 0.34921100551352935, 0.49004913935925043, 0.432542693822872, 0.51840547547479865, 0.0048294863960115508, 0.10735800737129741, 0.29601321937771352, 0.065114536328428341, 0.619179455681096, 0.39902660832583658, 0.894886076529092, 0.61793068205933577, 0.904384983760988, 0.50339400846029669, 0.89195431743719056, 0.72966676443254186, 0.18864113588467379], [0.510837178188414, 0.050851241666467484, 0.28442465098548908, 0.82417322635675327, 0.11299673914897179, 0.36046971621512769, 0.51205796709114448, 0.66761917185536568, 0.25924801537811126, 0.46747610082172786, 0.075751868639053366, 0.417899658739466, 0.017835147054534151, 0.5364567488079045, 0.79976731528571987, 0.99214819887466188, 0.86740719587880288, 0.13086695912257784, 0.83584211602622527, 0.922031913065875, 0.32926466584486569, 0.087835319690293567, 0.45443662576789678, 0.96080095120271691, 0.19144601980397902, 0.64600948804752578, 0.19095684882450137, 0.70566245630404956, 0.5728533274540859, 0.27801163400030726], [0.77728368563869421, 0.060582773303465065, 0.72893703993785919, 0.777705285630967, 0.14538181800014738, 0.791379878100802, 0.81002972551888275, 0.15735777533410567, 0.80245711330091174, 0.25797477626186638, 0.74923949108924448, 0.82091500916045468, 0.046077216440706725, 0.64792964887186555, 0.078883565072431439, 0.29698884916744528, 0.81286449013061379, 0.68090736489166359, 0.75276306577383434, 0.53878953112438455, 0.4358166258751397, 0.80544546452824861, 0.98615047603116879, 0.42811854847808894, 0.22300491979365511, 0.367445045193459, 0.71618134309389181, 0.6566704685854744, 0.56434590796903084, 0.09403301323320945], [0.53241208628156977, 0.60027471712462266, 0.22333716095732925, 0.50696411100162109, 0.83062640492685158, 0.15034108225284615, 0.57709347032964831, 0.556486677106241, 0.5025426668237456, 0.72165601342526153, 0.83071481069805364, 0.84960399281995824, 0.689895357695887, 0.81680887307635319, 0.65096302895655045, 0.33592101171909383, 0.0034336204504900047, 0.59491083971015712, 0.23941933417788464, 0.20977397944338083, 0.13570056950551224, 0.86590131222021416, 0.59673997749755925, 0.08909803039028541, 0.98102711896801353, 0.81162003517155912, 0.61732282767271018, 0.23949634123464703, 0.21699446757510632, 0.52095565225998386], [0.69764304363933849, 0.72623132400984125, 0.79868991456783378, 0.11876428171850673, 0.74171116896387967, 0.89736700685986859, 0.9704003406749856, 0.86688442682327971, 0.8035520524197548, 0.52804879590935438, 0.2089589358298638, 0.12045775741330678, 0.11929919630696029, 0.288915336441934, 0.97878029697171542, 0.43389741515493796, 0.95044314151513132, 0.94089461782601458, 0.30068952260281412, 0.42822900954821463, 0.48496903937595859, 0.80414392686868852, 0.91497175493206417, 0.25516183528921133, 0.024562985897052303, 0.60065225353092178, 0.25363790279137033, 0.62313063409363911, 0.82376272930663241, 0.72802989535747908], [0.527723699648675, 0.38668165406839161, 0.24087278831596093, 0.38626946164413589, 0.9825088472526472, 0.64295856221293435, 0.31066053161219964, 0.74976892719019261, 0.776863583047958, 0.12767745731550328, 0.45529368986375507, 0.5536664313027847, 0.0025000685784795085, 0.45786271193845229, 0.81267627065609949, 0.046543034531495775, 0.074726867753082793, 0.42974796956612737, 0.58028464764526932, 0.25080874451564394, 0.098374337218951724, 0.14762987275914652, 0.84780057536505182, 0.867593766729821, 0.840879742885282, 0.23309462941409798, 0.62328411343587919, 0.96472592398973878, 0.62909073916269764, 0.74392021613594894], [0.14532367994551953, 0.79438161784813077, 0.8271665570909329, 0.71809037693135824, 0.1175467019161498, 0.7012570888990286, 0.55002614548691819, 0.45073863497133926, 0.2020688508478502, 0.10261312595484451, 0.9201133091909105, 0.71047394745800707, 0.96490608550321555, 0.94830503617426753, 0.4848185191583253, 0.1064222826975858, 0.47906770866275383, 0.9019171922329241, 0.3777183617600689, 0.7284508952780494, 0.95999558004129593, 0.72337962158359159, 0.42886277922067195, 0.00042546285601285927, 0.13320855344729543, 0.97895110860912082, 0.7076614721641864, 0.4099825893228175, 0.5154572485891008, 0.25621219776718074]] [{"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}, {"rRkaro-fKHhpuG-jdK-tBXH":33282, "yTbZPG-XMGHdu6-LF7-tBaN":87927, "aG9Fdm-PiMW9Ia-PWv-4tJR":14259, "97c9xv-sAxOwyP-1U7-xMXy":24696}] +19 [[0.064110097878604777, 0.20880075639202911, 0.8271834124783779, 0.5409920484990518, 0.970148587111359, 0.8291199604200874, 0.044393498739233261, 0.91190206355989478, 0.19435805338209444, 0.6088772957950721, 0.12166195872864705, 0.7365113753875705, 0.60576251085398969, 0.62022100824175985, 0.74187300322342165, 0.94936690438807747, 0.78426231312474293, 0.31738627446919121, 0.98052773662963755, 0.54906669154999588, 0.7306481551870585, 0.91122866732966346, 0.12487398339355948, 0.76713968230753071, 0.618707215915597, 0.91979922806146863, 0.64390860648760961, 0.31141990354233229, 0.074803837115609118, 0.43820449995686428], [0.05293474877975668, 0.59874036347260573, 0.994730641275995, 0.23447163280404404, 0.081396131184527931, 0.22087379573971455, 0.963290508159028, 0.12286478403754542, 0.24899941617228583, 0.49799881271520341, 0.92898345288182749, 0.19207651745115994, 0.68267454084659507, 0.86557144183928347, 0.90433696334019287, 0.37294302319189276, 0.76571224271880556, 0.013334003612952228, 0.11183198945545036, 0.51881014778317891, 0.0004569537528418266, 0.14929425627582216, 0.17604121462071998, 0.64140260376639169, 0.28722216694833014, 0.49685988558224614, 0.67450990136140032, 0.19602575084654761, 0.67463485092073072, 0.14991470207922175], [0.603074291646202, 0.34215596272192583, 0.52040384759419211, 0.708474187158181, 0.0857255752807089, 0.91895189057390581, 0.91654722276353917, 0.69666027147459, 0.96479285559991146, 0.46994602307975308, 0.17940143420889043, 0.62508628933710408, 0.74413130365693714, 0.027151439601572402, 0.18227555661983275, 0.70134448008259032, 0.72616662825854039, 0.522046834532456, 0.31740141159713831, 0.61514069401297722, 0.0830606424543997, 0.629377632268738, 0.91387106119805128, 0.52872157297721034, 0.032926999696707537, 0.56943910885471549, 0.0095406552024532854, 0.3885919308106236, 0.25393991275602468, 0.89548319576965973], [0.47742011047200494, 0.9411209293277063, 0.1623431818122556, 0.57511707363094466, 0.64085594261605616, 0.38248756890809066, 0.704780431302352, 0.73803911149841284, 0.67685448933481718, 0.40251757330928051, 0.871032010348747, 0.30718307154211122, 0.30318136381722716, 0.49231521440410264, 0.22015750745550089, 0.35631904081039811, 0.47241232668956457, 0.77719493447143129, 0.33120762102615009, 0.44438584444104556, 0.054899540037757433, 0.30927004806412484, 0.23908239680883692, 0.36530566176635471, 0.78265636533937477, 0.708156078984015, 0.59677746753457339, 0.72175914552540033, 0.04745325032183878, 0.27359901496901207], [0.51717377791739383, 0.48237525655688296, 0.4989554469046531, 0.21043984552260597, 0.79069051212126851, 0.21401640277072165, 0.77252353213621916, 0.4041778558213156, 0.44953459219249192, 0.994316863228451, 0.091335689766844586, 0.26449793584983772, 0.45494455571465897, 0.1043581737903454, 0.93378046022119854, 0.51447727769773455, 0.62074085068850993, 0.47734207063856937, 0.43460752883904541, 0.44478937150257136, 0.75465796462090851, 0.53634846546186754, 0.697113005313418, 0.075074684970783845, 0.39834844899092925, 0.4927417479093652, 0.733674968503001, 0.86874831448750933, 0.47761299170478766, 0.39651601028638828], [0.32324410987579033, 0.13286740310971779, 0.6363241990528361, 0.23084601827524021, 0.6769330461494496, 0.0012818496596150863, 0.020676448279249193, 0.12533702730569662, 0.43663733031146312, 0.94714392784248791, 0.80480767798254416, 0.32594386545138976, 0.19666008862934847, 0.0999537937411834, 0.80450387408360757, 0.90425549248837023, 0.43460113064309625, 0.25266930375212981, 0.11260963324294726, 0.3148181981311291, 0.46531075160236068, 0.33601059966469848, 0.80793402700776029, 0.81639726014544056, 0.79049722928841037, 0.30994078480974552, 0.43885356661700503, 0.46155303641028744, 0.0066162080233014287, 0.87640608429899813], [0.16347908743370343, 0.82863441046388808, 0.25495927017780229, 0.15277043971102222, 0.14016667598422661, 0.92126898102274069, 0.050397781226727556, 0.46198066826227024, 0.00531771922674662, 0.69265574003552233, 0.20902782496421324, 0.69335035532955358, 0.04982629297481711, 0.4654429660836723, 0.510000053574241, 0.26695178882617565, 0.011196687721483523, 0.11270000143751002, 0.67310839245181631, 0.59495722781377347, 0.23526813710738237, 0.24146894051237078, 0.21125003669733478, 0.074871703605844275, 0.32964718580770425, 0.77760795030313856, 0.72565815612454987, 0.0724522405099296, 0.33036272564253433, 0.28911538197360531]] [{"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}, {"fSIAJB-L7j5loh-bLq-Owqr":7706, "obrnSB-HP7EJMp-RtS-loo9":44673, "9HjxXE-0Dq3mgL-QGj-sLMO":86885, "aRUGuy-UP5KD3v-mhR-HYRu":4467, "6Vm22d-LhQt61b-jLd-1JCV":7734, "J73UXT-cOArI6A-pIu-9uCq":33156, "tHFAII-tgyPATb-DR5-vDYZ":52365}] +20 [[0.684257829378934, 0.70388618730619446, 0.91669107239388337, 0.097070795145535627, 0.42518446096791107, 0.32749456014199685, 0.17573416201290015, 0.97077480892428736, 0.22136291388316398, 0.578723728596944, 0.87961023616505507, 0.64923384344922963, 0.17114415678257822, 0.32973463962151495, 0.81838256249274288, 0.88938390395559119, 0.016497830529647928, 0.43662933626524647, 0.70965984643835534, 0.44093262810597977, 0.054137428707956681, 0.075643253405023758, 0.83884039021903234, 0.48421763807034079, 0.71558418601253371, 0.80080476060084971, 0.40895931615416192, 0.16088512152326329, 0.512977371917522, 0.3160119616854663], [0.81306053344422713, 0.6322334642928813, 0.21247808463565032, 0.78357908124712439, 0.5976780287131096, 0.94479364139108335, 0.293402462909365, 0.60165572758977282, 0.97292634462827332, 0.608694274025095, 0.9787592554665383, 0.97452457245866653, 0.34286320304789053, 0.20158625315070788, 0.38165607394819356, 0.27263346016266221, 0.47920176529425895, 0.49937788664277916, 0.602077346789228, 0.13332110815448361, 0.64211402046436361, 0.60475846517080167, 0.92982618099858, 0.48854527817089943, 0.16378080450411325, 0.29546478418459932, 0.66196160881014632, 0.22252543622022014, 0.75350799299380133, 0.34647108880502431], [0.42685148051589594, 0.96006269938424182, 0.057642583526824609, 0.8937603749522689, 0.32200023400030153, 0.56540132741262372, 0.813186257077971, 0.0020607948815509758, 0.61910044711105006, 0.29184808292388476, 0.27091825240390865, 0.47490785004847624, 0.26729540466476842, 0.16396712948538539, 0.43200358606436717, 0.60261951447006967, 0.93442924198166954, 0.097019268564319083, 0.041475894127053325, 0.98224808803456742, 0.866100726904561, 0.46241681384594147, 0.21264981278387085, 0.76329891000290739, 0.41362492216945212, 0.29331601625089487, 0.018429804089057811, 0.67398285799902047, 0.68454689939623581, 0.09107091870250128], [0.37864853764453854, 0.96499617635454893, 0.403140180911294, 0.24112443558816066, 0.096980238171250788, 0.97129732173329941, 0.73336193107586489, 0.37601325994404866, 0.95663956764463864, 0.33707753951723041, 0.51302579347244093, 0.63351534545960053, 0.37099015868953789, 0.075098375878772, 0.075492478859039869, 0.649570208593355, 0.20335131191592548, 0.53879545170655263, 0.59396715519145882, 0.29845331232738515, 0.019704900198906916, 0.074025021005791, 0.67353201598651669, 0.64678838372401082, 0.21866937803459208, 0.58705602189814732, 0.73733091476191759, 0.32333201680011259, 0.19502109811402812, 0.51638678439856267], [0.60586266289748381, 0.523067090294932, 0.93620757235915, 0.72819265827760149, 0.55212829064626134, 0.16199295778747713, 0.22571140564280612, 0.66868670388186879, 0.030416473176450465, 0.2781018946857361, 0.93278761568790336, 0.70140254356106035, 0.22280102149269343, 0.63438289796068414, 0.70501088750510388, 0.93338514533242389, 0.5249625668914828, 0.621295376813799, 0.7482338445504918, 0.9915015843925834, 0.10173635218603483, 0.97900699278232062, 0.27797132536085234, 0.14856035771390796, 0.68282659677377766, 0.56379148886928965, 0.65688899335932893, 0.91865482085412753, 0.84041731187719071, 0.36833349111224023]] [{"4wNll1-xGgn5zl-3b9-Js3h":16418, "TkepGM-6zRAxkv-VRw-Q4aK":94366, "KVKuDx-5fekX4O-XlD-dxE6":27789, "4IhvxO-1SzMx79-6Kr-ZZ4B":2514, "nEwJeB-7Gl3iB3-my3-NMfo":32437, "BU5VhS-1VQynVx-4GG-B5Bw":95501, "H6xGTQ-0R5cYCC-1iD-LO74":95422}, {"4wNll1-xGgn5zl-3b9-Js3h":16418, "TkepGM-6zRAxkv-VRw-Q4aK":94366, "KVKuDx-5fekX4O-XlD-dxE6":27789, "4IhvxO-1SzMx79-6Kr-ZZ4B":2514, "nEwJeB-7Gl3iB3-my3-NMfo":32437, "BU5VhS-1VQynVx-4GG-B5Bw":95501, "H6xGTQ-0R5cYCC-1iD-LO74":95422}, {"4wNll1-xGgn5zl-3b9-Js3h":16418, "TkepGM-6zRAxkv-VRw-Q4aK":94366, "KVKuDx-5fekX4O-XlD-dxE6":27789, "4IhvxO-1SzMx79-6Kr-ZZ4B":2514, "nEwJeB-7Gl3iB3-my3-NMfo":32437, "BU5VhS-1VQynVx-4GG-B5Bw":95501, "H6xGTQ-0R5cYCC-1iD-LO74":95422}] +21 [[0.6282416298019271, 0.86423479871855158, 0.64051640054198133, 0.47154464676809205, 0.22303068332930998, 0.31937323748612734, 0.56317897125316452, 0.25979315710605622, 0.14789328251911638, 0.26376910076159166, 0.40027622345858727, 0.70243539501977659, 0.98536253767324122, 0.82519130249684947, 0.35414858262751348, 0.6894238443029882, 0.093520836477473668, 0.78103127732292077, 0.97859265893645231, 0.02078030359522598, 0.49122472672000439, 0.11970785905635695, 0.26029733668637822, 0.87121649622366948, 0.00987485616436512, 0.9452851742025481, 0.36314161781160315, 0.78151600114070763, 0.26339756075910892, 0.91588597870031863], [0.93711834637182, 0.3775132414333876, 0.66486215121134973, 0.96605287307654164, 0.14853023608370153, 0.28020116251981186, 0.80116553019631909, 0.27903036756071564, 0.74923800263843732, 0.089444711019594614, 0.96647414867264214, 0.347011044786007, 0.57012284020880721, 0.89897157213494616, 0.55062076628953149, 0.4857093069925712, 0.79755842258645682, 0.53175841448450578, 0.28041097138186966, 0.35419645675568989, 0.27157039815586781, 0.57262462816521964, 0.019803147546250788, 0.87175142830743779, 0.83049422282467777, 0.31879215504049763, 0.94918249153744649, 0.42055161544818587, 0.41932485296844813, 0.96936449307543981], [0.46445778635382695, 0.376348349085758, 0.27758479315000906, 0.73112364682219255, 0.06346787081075278, 0.81884234804889611, 0.02741964989153467, 0.030739039871592877, 0.82019167722019026, 0.739294530091044, 0.89376329405911459, 0.69242770887066, 0.78651453575247321, 0.18497779019765304, 0.60579747382531945, 0.76530013473365777, 0.8068104125596, 0.55839568573179521, 0.752563560629487, 0.66607223565652529, 0.78235797313141675, 0.72376478512783, 0.49546678383728038, 0.44051804042267451, 0.38061632271804957, 0.664076727114492, 0.576546441759269, 0.24098419233754775, 0.54864352494916868, 0.17912749630498559], [0.50376803483459054, 0.56748175676439983, 0.96044937085593962, 0.51822081650273988, 0.64322637732353216, 0.426316827066875, 0.70483156031431538, 0.90082351621953916, 0.70303118057875758, 0.32291607515588994, 0.10325186881001525, 0.59427054947880986, 0.75477992850084863, 0.92444940070160253, 0.53656725607365885, 0.58968801103300661, 0.50590532579278036, 0.30304670891192642, 0.55493844140972881, 0.94562264245697025, 0.0098171904877336624, 0.90185084229468115, 0.3469959983057106, 0.55343669944455587, 0.38064229290607909, 0.75384514811555658, 0.85235240222757336, 0.41934726850472603, 0.36549742287518483, 0.027338146432063337], [0.41580589734420359, 0.55505945134656842, 0.81961471041716616, 0.67218377377521588, 0.90208976707286914, 0.22226339676962148, 0.22827025158799108, 0.79726127097541788, 0.6167787935153618, 0.58583511837145341, 0.42435051507918276, 0.42243308111838085, 0.21197223957190403, 0.7964013604241047, 0.56152353179187608, 0.41714555243798113, 0.99463049374802381, 0.1449995532617534, 0.22036460154813964, 0.76303200130963778, 0.70095037279216754, 0.21088373275463257, 0.24841929272437535, 0.75932267484755733, 0.85192207072577786, 0.85809746741869153, 0.56506920622518075, 0.42756258282724557, 0.263218129494871, 0.52725614608944349]] [{"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}, {"9XvZPd-kwVEYMj-gE6-dNIz":34040, "oca7be-h5xFLZH-qm1-15w7":35442, "5Lczct-ErXeaso-iYP-ofU7":29334}] +22 [[0.86422759673630334, 0.92679339642157155, 0.039232949953334928, 0.36945170719753029, 0.10445499958631888, 0.6693224249600398, 0.73125389499857574, 0.75744906617478513, 0.71545217427979924, 0.805416660525261, 0.89630901914123173, 0.71948614477196726, 0.93691749542168989, 0.67053738008471653, 0.62821235526745189, 0.89034704938005937, 0.56893543616380793, 0.73804662755730976, 0.96969112113386746, 0.31105666569182355, 0.08005316545771668, 0.83085210167034584, 0.63359136988200426, 0.62968064023686565, 0.42734081046608352, 0.76956128031931081, 0.0209216397085068, 0.37488063142077233, 0.768153467069212, 0.40566140451337573], [0.33312799118303016, 0.35669801202424678, 0.2364769512392908, 0.77046887648246265, 0.20978371145701358, 0.29689189587181308, 0.48115340111003735, 0.63406806860461229, 0.98791526698725718, 0.12025822906936579, 0.71894726815194143, 0.11318595442127843, 0.37547576573936137, 0.94870887498061351, 0.64232529214203449, 0.092925762474396212, 0.059015698039078357, 0.73149012192975282, 0.63998958309159992, 0.13450930276689155, 0.52439029747867316, 0.61777327912638236, 0.43761562915212482, 0.22965408194558845, 0.15938169651995826, 0.63194962181173242, 0.44691184504341364, 0.23193811975661049, 0.52840275661339331, 0.94731530699985944], [0.82303138422101729, 0.046481731067986587, 0.56198950498846534, 0.070432032690099788, 0.47353040452644, 0.69150226435611728, 0.287576903683485, 0.546301803597882, 0.036481768396258052, 0.15877417375630598, 0.64309274082595425, 0.3735951123717961, 0.15101683105489261, 0.59475344362915217, 0.86119308906096514, 0.17952353112140396, 0.18048177517783759, 0.70010208837915067, 0.88950173498820906, 0.34554964213432826, 0.91655981215967952, 0.40257485835146645, 0.32100533591508984, 0.69797800330235826, 0.69212377634289368, 0.79670667814962659, 0.34879503130374911, 0.081573467940392774, 0.761248169591419, 0.063466367410725311], [0.33063387068701855, 0.11970372646243921, 0.7667236918810777, 0.584018311126053, 0.65136943118594592, 0.73110523837403241, 0.60858135987695661, 0.74396061103275657, 0.70499895881252794, 0.52122269270996457, 0.051547305863129167, 0.98788455012400533, 0.98323434650262143, 0.99517435129100673, 0.96501357680694977, 0.52882699691363111, 0.45370266711586182, 0.64974663330176541, 0.17602383861009174, 0.743163934589438, 0.638457374461502, 0.39342299905257927, 0.53393793478525708, 0.68871944378339145, 0.062264522371997866, 0.4163040489396217, 0.93485214582003984, 0.37234180570211528, 0.030166700793004653, 0.58587686158830632], [0.73451675376014947, 0.3629241680827231, 0.01511748629473908, 0.67994360698457457, 0.19040663266800828, 0.096824231285298157, 0.28901713333001955, 0.28462059792734917, 0.31676420860124077, 0.754863937269606, 0.81427040271242823, 0.98317075099528051, 0.81084848450399138, 0.90356649315426707, 0.82802555853068194, 0.54053429056312319, 0.5142879075345429, 0.18801481466692238, 0.9975431807265, 0.1487731200243203, 0.21034880297580372, 0.239692171127886, 0.50268240697696009, 0.44024722425880569, 0.42609516331063912, 0.61989178211515383, 0.91028102607686034, 0.89819166033152853, 0.3078297781928927, 0.057759472600305117], [0.24524525567117861, 0.49846525051606372, 0.56887057387244688, 0.46175951121030345, 0.23445390255292298, 0.9015932430150958, 0.97063021008641293, 0.18433046191739422, 0.3021206837767263, 0.28082775423607109, 0.791131056177174, 0.80172443842317853, 0.52904751332254885, 0.91980132004222093, 0.25950157663687567, 0.23449993531255453, 0.45324084490119443, 0.042450928280596933, 0.99066454015700622, 0.37886442233074935, 0.28565196166597862, 0.94824060354341833, 0.041340939772944174, 0.54504006503307656, 0.842341029030388, 0.41731311169191165, 0.326658548429516, 0.90605222021210585, 0.2531545672515183, 0.52305220399471131], [0.7860668493597851, 0.848816805903052, 0.29704402901094373, 0.44363874081709842, 0.86792517421613757, 0.58912926340540506, 0.035442811175551481, 0.59953550272445766, 0.61404464783925339, 0.66313479145103682, 0.26765827759399241, 0.46043707887745366, 0.86862813918731141, 0.94008543968710745, 0.21602375924377126, 0.66307279948826914, 0.91046729868741982, 0.61466651168279607, 0.76359533896960019, 0.66035204226056188, 0.11291897268386697, 0.2319017718518287, 0.51346642100394135, 0.98269327131909112, 0.073098074196493612, 0.17362226865932928, 0.91631335572597938, 0.33397916241585879, 0.019887703460604533, 0.19271184231513572], [0.72157286501585816, 0.16743220143951654, 0.20978120339490547, 0.51981714154476266, 0.047692713166035849, 0.090806277302752592, 0.45023521716035164, 0.78693823451942435, 0.43818479373798391, 0.43564194738817585, 0.85212695717363707, 0.5132910475023732, 0.37274988861589253, 0.59193380500760051, 0.40607249930110345, 0.18106044446223668, 0.83653152517629081, 0.25741082321411513, 0.0017772639819795266, 0.99213853988526579, 0.7155961872422133, 0.57973124563267353, 0.065254118150196883, 0.24167031706031572, 0.98302574939866139, 0.280246074318359, 0.075658087291649134, 0.87020400541687182, 0.86754710104496635, 0.23958580132631258], [0.841651234323355, 0.1634713936135328, 0.45541323802838185, 0.69817739546805446, 0.56662491896276346, 0.75729905826998412, 0.46628892548722722, 0.17889966369985921, 0.41277102209597283, 0.074878247433725509, 0.26518298657878392, 0.47443689489090934, 0.48590759735886213, 0.074598053509818851, 0.52500883981523694, 0.5584920000615945, 0.906128467267314, 0.71887229076012948, 0.33814349835599355, 0.077872245547963459, 0.90376066278344735, 0.88786812292904438, 0.99039437772516037, 0.16156201634991829, 0.68577689757377081, 0.55677006297121689, 0.057072367794676615, 0.64324871042557208, 0.15472555594277382, 0.93118997611523713], [0.95840903832214175, 0.43536926975703194, 0.430403322250562, 0.89256053536868962, 0.62034422375322784, 0.71559680529637848, 0.523910879457945, 0.54382547489793021, 0.42674982246707327, 0.79078760147477567, 0.44439518260718314, 0.86546725627892351, 0.22044451100825935, 0.98876145715351016, 0.480696871668852, 0.49288869340229291, 0.71289327309331885, 0.69488867628937012, 0.39328428474217891, 0.40839214741909491, 0.29627438654167126, 0.69830928020674454, 0.77627069273262239, 0.90447625578977109, 0.91663608026029064, 0.34171180493051734, 0.90497470252706513, 0.51989661881109317, 0.36315356065964988, 0.5789497158861695]] [{"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}, {"kpKSB7-aJSq77c-FJj-b427":80319, "ZFjwkJ-a0lI0hb-tJY-yE9B":91890, "3ot0Id-UkSkrXQ-miG-Paik":48776}] +23 [[0.014350715683691062, 0.31198088124604328, 0.44919313540445149, 0.69403977782546622, 0.87170873297793683, 0.66908538287752817, 0.92408997060257614, 0.040486850653616457, 0.9018635145076721, 0.0769809553877272, 0.6379446662337267, 0.3287831700120436, 0.54633922249848688, 0.37187021408335308, 0.46124694626748486, 0.26703904141878221, 0.074211030034582182, 0.28004057053656206, 0.35419094952035479, 0.0611746267096257, 0.15039656641669574, 0.35241087496641776, 0.87091663563131516, 0.655690048743426, 0.4252383900625164, 0.63900437095912066, 0.56744240231480558, 0.31172800797857469, 0.009055027885049638, 0.80516072502532676], [0.64425221531667176, 0.53918333416015907, 0.4814716804509771, 0.43524522521595321, 0.26875802158795148, 0.89668378686629169, 0.77772399763046052, 0.46980845599732668, 0.54336646433206393, 0.32658683862800919, 0.892761466064651, 0.078595178647893893, 0.011636068226680618, 0.50170772142855913, 0.84885962119634462, 0.784496405052977, 0.92801970707433112, 0.1263139763491159, 0.78358299457110114, 0.32248392675513227, 0.25594430926225065, 0.25599223454606623, 0.33232390845787041, 0.54276193987254429, 0.50448775309556837, 0.479313334870065, 0.703274459344789, 0.64881589002654583, 0.065671113558733563, 0.23879201515861392], [0.97257248000802132, 0.1987332811463246, 0.67239133934080242, 0.6594578965984278, 0.16349449283887907, 0.59211761525201723, 0.96127849129794118, 0.00042479202066081356, 0.61595201933043942, 0.21511090824897638, 0.89526815215429234, 0.24816534564842063, 0.0011800208080797736, 0.02248611448865534, 0.97743851404921189, 0.4297689053594147, 0.091389039842109665, 0.09846596578413247, 0.77066889750780365, 0.94028041670728135, 0.055286058525905957, 0.65686997847235162, 0.22693428771683333, 0.55028844127461662, 0.61775083455617652, 0.016386764595307635, 0.72555854890279126, 0.7882509317926637, 0.89558834841162038, 0.31821410661359062], [0.52150662458084041, 0.076617680921092668, 0.10116013549156622, 0.14100456397137329, 0.51280381671674746, 0.34162834003161624, 0.59183027864920568, 0.711999525290495, 0.85042796670357323, 0.81373577087425031, 0.93554468585583184, 0.11343025763383863, 0.93452858684384066, 0.38892429457273858, 0.761495272786269, 0.73970947185415326, 0.76758431203696176, 0.23076300164114683, 0.1697028125436405, 0.2736326361441469, 0.46811803712397371, 0.25885449154401119, 0.38448599303768227, 0.61875298509121024, 0.98784704568021076, 0.79140986148762749, 0.15416634353835035, 0.55029010908854792, 0.72065311235516083, 0.88800080597899467]] [{"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}, {"mRcyhm-zVPP0OI-BbN-vbWp":69926, "ItCvx5-WMokrrO-Iix-H8E3":4825, "T0dUvQ-suLmAEG-ei0-J8kw":71316, "eEvNr5-eH5LT8V-0sx-uwmq":4401, "DABalM-wRWysV6-RkU-L3iu":25873, "2xv9Wk-HOnCyFz-qJo-DFwA":56559, "GtMcg1-Qyd1Bhj-TQC-YY8m":99475, "bSWYm1-ihLnYBs-Aoz-oW9w":47807}] +24 [[0.61184783339368509, 0.90726494717810446, 0.052551004524158063, 0.68531092865850463, 0.075755468884431854, 0.23243887205627478, 0.016006660882104895, 0.55489866095327578, 0.35372039824367252, 0.21556295384126856, 0.6549740705467787, 0.023472470202435036, 0.59044348569592109, 0.49133437377913369, 0.14329430604691828, 0.19984437254013632, 0.59585062365578012, 0.4471476134147464, 0.00507627405204647, 0.92821258175716037, 0.489854027217573, 0.69896669376449327, 0.001897647243347178, 0.36595913422938275, 0.22067223054372076, 0.60192743816314065, 0.575342607152971, 0.53442522254210534, 0.068556167014707681, 0.86772174994276641], [0.86346193568348206, 0.3722357098466762, 0.44305710539417176, 0.4711672727591244, 0.39665428026587823, 0.446720870385205, 0.50951156438126033, 0.41996791797408417, 0.19663240219672462, 0.88621008165500126, 0.43920722215859553, 0.63452145426464157, 0.15155144254304354, 0.28444860782758485, 0.36773319976999186, 0.13873293781487994, 0.22104077475690886, 0.29811544774116683, 0.46730675018572565, 0.99814347686572469, 0.23679509946317856, 0.40939629685862267, 0.89140766983918107, 0.95828565228061169, 0.80754416418043173, 0.85779277916454377, 0.73384093609018886, 0.53402885775548492, 0.11163475307662785, 0.058825334716302025], [0.559386849562951, 0.37977868726594954, 0.1661228975089587, 0.92201375316275991, 0.50948735280022583, 0.13074681650034548, 0.49818769808097907, 0.35649605477937674, 0.46395567135083093, 0.54710854463691816, 0.012843003147055332, 0.909872701998225, 0.24848652836307605, 0.71571766544974869, 0.25033647687220328, 0.92434013832638884, 0.41778186587660771, 0.26814910569761474, 0.15729018803863293, 0.81390356510356, 0.71539755520156822, 0.059191650184280187, 0.95755070241850959, 0.067436227847928465, 0.91395418359407221, 0.516514978993087, 0.70188088429268591, 0.36457820843832045, 0.37314540182784117, 0.92008583208863393], [0.60003925291313032, 0.75494502573079136, 0.31801523221235706, 0.19901093622015298, 0.29445737690556495, 0.19584237438657137, 0.53626578847805551, 0.44383496811119982, 0.82163953134096079, 0.86966015095681, 0.92089506912885821, 0.22392350492929813, 0.12557267194665045, 0.60934545474434942, 0.0432541922118278, 0.97241309155540723, 0.63684962255191624, 0.44883085137049827, 0.731652123676696, 0.15531120765858852, 0.61187142395067307, 0.78502037247831091, 0.45192513351289409, 0.67668141368847756, 0.63840975040328485, 0.88891310492860665, 0.13892737211786843, 0.988449538534481, 0.95697364731657208, 0.53566975735402167], [0.011687855449314721, 0.67121939157375166, 0.021839500863293115, 0.33690505102353496, 0.82001818377685964, 0.8315869807159525, 0.031266837552144322, 0.95321478307157348, 0.91664479329888826, 0.52179478307792182, 0.9933365952399702, 0.74092323313963782, 0.52647793260921183, 0.52916831217902072, 0.26201308560334, 0.54426290316104708, 0.59341914686376307, 0.3056293034913341, 0.3774439252966888, 0.28267573549539726, 0.625147978864336, 0.51168143616285544, 0.47517756361610075, 0.70265047329589636, 0.027617528696819549, 0.63461563765746709, 0.49089083558414481, 0.16448865767759757, 0.52352776495978437, 0.40728402102745453], [0.88038413835515, 0.61661699646567436, 0.38283429103645572, 0.30164277222703961, 0.59901492516589794, 0.9735419130629287, 0.35838974198496687, 0.20036616349234082, 0.64040500471769024, 0.9582667625993363, 0.41610348232060579, 0.46018508155788806, 0.52117082345381427, 0.095970891890699628, 0.57654505721604965, 0.9722053227533527, 0.71659001361003394, 0.95297437321138312, 0.44370490941675622, 0.49731215422780839, 0.20153177274627521, 0.90349004183731529, 0.286666368882732, 0.508869776699888, 0.931642281151314, 0.67639051125794625, 0.8612302936916908, 0.010275009748340391, 0.31169083726394564, 0.68519457168007014], [0.51107420285637506, 0.31152695948838116, 0.44114632716700986, 0.67670379857456531, 0.4718161535199219, 0.54682544657752807, 0.2014674297149337, 0.547196859892135, 0.71639265870663671, 0.79431267092405267, 0.055254711538245793, 0.26981554066701563, 0.12644925979350752, 0.019424387086474204, 0.76432574421613564, 0.75275327456739016, 0.25198125977569574, 0.7049177776770329, 0.10193978144804283, 0.10508302986358264, 0.43025118527350381, 0.9534957743132052, 0.03381127089033753, 0.22553835159543023, 0.57504876216779888, 0.053048165816900972, 0.77321656514571069, 0.84928816934723694, 0.51864804021491151, 0.97988196811505168], [0.57741164568974024, 0.18872173201159081, 0.3913554416317343, 0.081914855264381026, 0.14574304529094839, 0.086487379648751528, 0.15733180334665597, 0.2316572030036429, 0.0042480033148998286, 0.16250505131147908, 0.090409978707767236, 0.067909715269275028, 0.31720418277496709, 0.78139905093031337, 0.66991933318788111, 0.55545507762545943, 0.15300315383497387, 0.79277417420428153, 0.10406709603819286, 0.98362392248137021, 0.34988590172846257, 0.90398486771912157, 0.44721540172686947, 0.69746664562262584, 0.77649290734045828, 0.23352361741704009, 0.067432641907749669, 0.30936453366817529, 0.9628451607445907, 0.062881291772265535]] [{"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}, {"NBapTq-WIv3Wqa-V2x-v5zf":37362, "auiDf0-ijlJN3m-ko3-Q4CH":79463, "IvQJFx-wxfDGca-OfL-JOAy":45961, "3nL7DF-XtKfU5W-xHD-Gaz5":61262}] + +-- !sql -- +0 + diff --git a/regression-test/data/javaudf_p0/test_javaudf_with_decimal.out b/regression-test/data/javaudf_p0/test_javaudf_with_decimal.out new file mode 100644 index 000000000000000..4a05a63689f3aac --- /dev/null +++ b/regression-test/data/javaudf_p0/test_javaudf_with_decimal.out @@ -0,0 +1,13 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_1 -- +[1.123, 1.123] 3 +[2.123, 2.123] 3 + +-- !select_2 -- +{1.123:1.123457} 9 +{2.123:2.123457} 9 + +-- !select_3 -- +1 {1.0000000000:1.1110000000} +2 {2.0000000000:2.2220000000} + diff --git a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql.csv b/regression-test/data/load_p0/http_stream/test_http_stream.csv similarity index 95% rename from regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql.csv rename to regression-test/data/load_p0/http_stream/test_http_stream.csv index 3b4e1ad6dc800a6..d5df966cdaacc96 100644 --- a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql.csv +++ b/regression-test/data/load_p0/http_stream/test_http_stream.csv @@ -6,6 +6,6 @@ 10005,ff,西安,0,66,8888,64562,4356,9.133,23423.45,666.666,666.666,2017-10-06,2017-10-06,2017-10-06 12:00:15,2017-10-06 12:00:15 10006,gg,深圳,1,77,-7777,-12313342,34534,8.100,12,777.777,777.777,2017-10-07,2017-10-07,2017-10-07 13:20:22,2017-10-07 13:20:22 10007,hh,杭州,0,88,6666,314234,43535356,34.124,324,888.888,888.888,2017-10-08,2017-10-08,2017-10-08 14:58:10,2017-10-08 14:58:10 -10008,ii,上海,1,99,-5555,1341,23434534,342.120,34234.1,999.999,999.999,2017-10-09,2017-10-09,2017-10-09 25:12:22,2017-10-09 25:12:22 +10008,ii,上海,1,99,-5555,1341,23434534,342.120,34234.1,999.999,999.999,2017-10-09,2017-10-09,2017-10-09 23:12:22,2017-10-09 23:12:22 10009,jj,南京,0,11,4444,-123,53623567,11.22,324.33,111.111,111.111,2017-10-10,2017-10-10,2017-10-10 16:25:42,2017-10-10 16:25:42 10010,kk,成都,0,22,-3333,12314,674567,13,45464.435,222.222,222.222,2017-10-11,2017-10-11,2017-10-11 17:22:24,2017-10-11 17:22:24 \ No newline at end of file diff --git a/regression-test/data/load_p0/http_stream/test_http_stream.out b/regression-test/data/load_p0/http_stream/test_http_stream.out new file mode 100644 index 000000000000000..4471c47db0ab1a1 --- /dev/null +++ b/regression-test/data/load_p0/http_stream/test_http_stream.out @@ -0,0 +1,571 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +10000 aa +10001 bb +10002 cc +10003 dd +10004 ee +10005 ff +10006 gg +10007 hh +10008 ii +10009 jj +10010 kk + +-- !sql2 -- +10000 4444 aa 5555555 111.111 3.14 +10001 3333 bb 666 222.222 5.32 +10002 2222 cc 453 333.333 4.321 +10003 1111 dd -3241 444.444 1.34 +10004 -9999 ee 21342 555.555 1.22 +10005 8888 ff 64562 666.666 9.133 +10006 -7777 gg -12313342 777.777 8.1 +10007 6666 hh 314234 888.888 34.124 +10008 -5555 ii 1341 999.999 342.12 +10009 4444 jj -123 111.111 11.22 +10010 -3333 kk 12314 222.222 13.0 + +-- !sql3 -- +10000 aa 2017 10 1 +10001 bb 2017 10 2 +10002 cc 2017 10 3 +10003 dd 2017 10 4 +10004 ee 2017 10 5 +10005 ff 2017 10 6 +10006 gg 2017 10 7 +10007 hh 2017 10 8 +10008 ii 2017 10 9 +10009 jj 2017 10 10 +10010 kk 2017 10 11 + +-- !sql5 -- +10000 aa 2017-10-01 0 99999 +10001 bb 2017-10-02 0 99999 +10002 cc 2017-10-03 0 99999 +10003 dd 2017-10-04 0 99999 +10004 ee 2017-10-05 0 99999 +10005 ff 2017-10-06 0 99999 +10006 gg 2017-10-07 0 99999 +10007 hh 2017-10-08 0 99999 +10008 ii 2017-10-09 0 99999 +10009 jj 2017-10-10 0 99999 +10010 kk 2017-10-11 0 99999 + +-- !sql6 -- +10000 aa 北京 false 11 4444 5555555 41232314 3.14 123.3423 111.111 111.111 2017-10-01 2017-10-01 2017-10-01T06:00 2017-10-01T06:00 +10001 bb 北京 false 22 3333 666 2768658 5.32 123111.3242 222.222 222.222 2017-10-02 2017-10-02 2017-10-02T07:00 2017-10-02T07:00 +10002 cc 北京 true 33 2222 453 5463456 4.321 11111.23423 333.333 333.333 2017-10-03 2017-10-03 2017-10-03T17:05:45 2017-10-03T17:05:45 +10003 dd 上海 true 44 1111 -3241 -45235 1.34 54626.324 444.444 444.444 2017-10-04 2017-10-04 2017-10-04T12:59:12 2017-10-04T12:59:12 +10004 ee 成都 false 55 -9999 21342 4513456 1.22 111.33 555.555 555.555 2017-10-05 2017-10-05 2017-10-05T11:20 2017-10-05T11:20 +10005 ff 西安 false 66 8888 64562 4356 9.133 23423.45 666.666 666.666 2017-10-06 2017-10-06 2017-10-06T12:00:15 2017-10-06T12:00:15 +10006 gg 深圳 true 77 -7777 -12313342 34534 8.1 12.0 777.777 777.777 2017-10-07 2017-10-07 2017-10-07T13:20:22 2017-10-07T13:20:22 +10007 hh 杭州 false 88 6666 314234 43535356 34.124 324.0 888.888 888.888 2017-10-08 2017-10-08 2017-10-08T14:58:10 2017-10-08T14:58:10 +10008 ii 上海 true 99 -5555 1341 23434534 342.12 34234.1 999.999 999.999 2017-10-09 2017-10-09 2017-10-09T23:12:22 2017-10-09T23:12:22 +10009 jj 南京 false 11 4444 -123 53623567 11.22 324.33 111.111 111.111 2017-10-10 2017-10-10 2017-10-10T16:25:42 2017-10-10T16:25:42 +10010 kk 成都 false 22 -3333 12314 674567 13.0 45464.435 222.222 222.222 2017-10-11 2017-10-11 2017-10-11T17:22:24 2017-10-11T17:22:24 + +-- !sql7 -- +10000 aa 西安 22 0 1234567 陕西西安 2016-02-21T07:05:01 +10000 aa 北京 21 0 1234567 北京 2017-03-11T06:01:02 +10001 bb 上海 20 1 1234567 上海 2012-05-22T12:59:12 +10001 bb 天津 33 1 1234567 天津 2019-01-11T17:05:45 +10002 bb 上海 20 1 1234567 上海 2013-06-02T12:59:12 +10003 cc 广州 32 0 1234567 广东广州 2015-08-12T11:25 +10003 cc 广州 32 0 1234567 广东广州 2014-07-02T11:20 +10004 dd 杭州 47 0 1234567 浙江杭州 2017-11-23T13:26:22 +10004 dd 深圳 33 1 1234567 广东深圳 2016-12-01T14:04:15 +10005 dd 深圳 19 0 1234567 广东深圳 2018-10-03T12:27:22 +10005 ee 成都 21 1 1234567 四川成都 2019-09-03T11:24:22 + +-- !sql8 -- +10000 aa 西安 22 0 1234567 陕西西安 2016-02-21T07:05:01 +10001 bb 上海 20 1 1234567 上海 2012-05-22T12:59:12 +10002 bb 上海 20 1 1234567 上海 2013-06-02T12:59:12 +10003 cc 广州 32 0 1234567 广东广州 2015-08-12T11:25 +10004 dd 杭州 47 0 1234567 浙江杭州 2017-11-23T13:26:22 +10005 dd 深圳 19 0 1234567 广东深圳 2018-10-03T12:27:22 +10005 ee 成都 21 1 1234567 四川成都 2019-09-03T11:24:22 + +-- !sql9 -- +10000 aa 西安 22 0 1234567 陕西西安 2016-02-21T07:05:01 +10001 bb 上海 20 1 1234567 上海 2012-05-22T12:59:12 +10002 bb 上海 20 1 1234567 上海 2013-06-02T12:59:12 +10003 cc 广州 32 0 1234567 广东广州 2015-08-12T11:25 +10004 dd 杭州 47 0 1234567 浙江杭州 2017-11-23T13:26:22 +10005 dd 深圳 19 0 1234567 广东深圳 2018-10-03T12:27:22 +10005 ee 成都 21 1 1234567 四川成都 2019-09-03T11:24:22 + +-- !sql10 -- +1500 + +-- !sql11 -- +10000 aa +10001 bb +10002 cc +10003 dd +10004 ee +10005 ff +10006 gg +10007 hh +10008 ii +10009 jj +10010 kk + +-- !sql12 -- +10000 aa +10001 bb +10002 cc +10003 dd +10004 ee +10005 ff +10006 gg +10007 hh +10008 ii +10009 jj +10010 kk + +-- !sql13 -- +丁伟 553 +丁娜 566 786 +丁娟 717 810 +丁芳 869 987 +万军 796 +万敏 552 624 +万杰 701 +万涛 591 +乔刚 654 +乔勇 611 +乔敏 932 +乔杰 613 +乔洋 978 +乔秀英 688 +乔艳 966 +乔芳 837 +乔霞 692 +于桂英 829 +任丽 817 +任伟 657 +何娜 993 +何娟 925 +何敏 926 +何明 1032 +何霞 664 +余军 931 +余明 891 +余芳 918 +侯敏 976 +侯明 807 +侯洋 878 +侯芳 661 +侯超 808 +侯静 800 +傅涛 574 +傅秀兰 568 1014 +刘伟 938 +刘刚 1040 +刘明 632 +刘洋 621 890 957 +刘磊 953 +刘秀英 679 782 +刘芳 715 802 +刘霞 615 +卢丽 979 +卢伟 955 +卢强 903 +卢敏 972 +卢艳 888 +卢芳 723 +史强 1013 +史敏 863 +史秀英 892 +史芳 730 +史静 689 +叶伟 877 +叶磊 980 +叶秀兰 986 +叶静 651 +吕军 866 +吕娜 716 885 +吕娟 907 +吕霞 554 935 +吴刚 872 +吴娟 852 +吴强 901 +吴明 740 +吴杰 577 +吴洋 909 +吴超 848 +周娜 1016 +周娟 964 +周芳 586 +周超 617 +唐丽 816 +唐军 930 +唐强 601 +夏娜 705 +姚娟 988 +姚敏 722 +姚涛 830 +姚秀兰 1005 +姚秀英 560 +姚芳 999 +姜刚 645 +姜强 619 +姜明 618 +姜洋 592 +姜涛 642 +姜超 951 +孔丽 883 +孔刚 676 +孔涛 1015 +孔磊 550 +孔芳 762 +孔静 846 +孙军 1029 +孙娟 647 +孙涛 894 +孙艳 984 +孙超 971 +孟明 545 +孟桂英 656 +孟磊 556 565 +宋勇 612 +宋娜 752 +宋敏 820 +宋洋 726 771 +宋秀英 777 +宋芳 765 +宋静 871 +尹伟 725 +尹敏 803 +尹杰 908 +尹涛 747 942 +尹霞 587 +崔军 563 567 +崔娟 794 +崔敏 811 +崔杰 646 +崔洋 589 842 +崔涛 840 973 +常军 857 +常敏 860 +常磊 643 +康刚 569 +康明 927 +康杰 635 +康秀兰 936 +康霞 941 +廖杰 780 +廖秀兰 823 1038 +张秀英 822 912 +张超 859 +张霞 779 +彭军 835 +彭敏 1002 +彭杰 588 889 +彭涛 555 +彭秀英 541 +彭芳 792 +彭霞 757 +徐丽 585 +徐娟 742 +徐平 963 +徐强 754 +徐秀兰 904 +徐超 910 +戴伟 874 +戴娜 868 +戴明 681 +文芳 637 +方军 606 707 +方娟 905 +方敏 573 599 +方芳 1030 +易伟 626 +易娜 548 +易秀英 865 +易超 670 +易静 736 +曹洋 776 +曹超 965 +曾强 728 956 +曾洋 659 +曾涛 789 +曾静 653 751 +朱伟 790 +朱涛 749 +朱秀兰 1039 +朱艳 870 +李娟 884 +李敏 695 +李洋 547 +李艳 570 +李静 672 744 +杜丽 590 +杜勇 962 +杜娟 687 +杜平 916 +杜敏 921 +杨敏 711 +杨秀兰 969 +杨秀英 1036 +杨超 856 +杨静 551 +林丽 766 +林勇 572 +林娟 640 +林涛 1031 +林静 684 +梁平 610 +梁明 818 854 +梁秀英 668 +梁芳 737 +武勇 937 +武超 791 +武霞 631 +段军 763 +段娟 581 +段平 663 +段涛 1027 +段秀英 880 +段芳 721 989 +毛勇 686 844 +毛娟 849 +毛强 633 +毛明 933 +毛杰 544 665 +毛桂英 861 +江明 542 +江磊 1023 +江超 616 +汤敏 785 +汤磊 876 +汤秀兰 583 +汤芳 622 660 +汤超 772 +汪丽 873 +汪伟 968 +汪勇 674 +汪洋 558 +汪秀兰 944 +沈明 682 +沈涛 838 949 +沈秀兰 576 +潘丽 667 +潘勇 961 +潘强 855 +潘涛 994 +潘秀英 600 +熊军 1004 +熊洋 561 770 +熊涛 867 952 +王平 1006 +王秀英 858 +田丽 985 +田勇 896 +田平 768 +田强 970 +田明 758 +田杰 1018 +田桂英 639 +田洋 977 +田超 753 +白刚 991 +白静 562 +石伟 578 +石刚 718 +石娟 697 +石洋 821 +石静 1009 +秦娜 833 +秦娟 784 +秦平 813 +秦秀英 712 +秦静 708 +程娜 724 +程平 605 +程强 998 +程敏 995 +程桂英 1020 +程洋 575 +罗丽 887 +罗刚 727 920 +罗娟 761 +罗明 597 764 +罗艳 595 1024 +罗芳 929 +胡明 864 +胡秀英 732 +苏丽 1022 +苏军 1011 +苏娜 741 +苏娟 690 +苏秀兰 806 +范艳 571 +范芳 781 +范霞 841 967 +范静 939 +萧勇 886 +萧涛 743 +董强 825 +董涛 1028 +董秀兰 696 1034 +董秀英 827 +董霞 729 +蒋军 683 +蒋磊 1035 +蒋秀兰 609 +蒋静 678 +蔡军 824 +蔡勇 798 +蔡杰 702 +蔡涛 862 +蔡静 1021 +薛娟 975 +薛磊 755 +薛艳 704 +袁伟 946 +袁明 671 +袁秀英 1000 +袁静 992 +许勇 693 897 960 +许娜 608 +许强 546 596 648 769 +许杰 799 +谢伟 1037 +谢军 928 +谢刚 893 +谢敏 673 +谭刚 923 +谭涛 559 638 +谭秀英 580 +贺娜 783 +贺平 911 +贺超 836 +贾丽 614 +贾勇 943 +贾明 917 +贾艳 710 +赖勇 652 +赖娟 958 +赖芳 666 +赖霞 940 +赵丽 934 +赵洋 629 +赵芳 746 +邓军 706 +邓涛 1003 +邓芳 881 +邓静 644 +邱伟 797 +邱平 658 +邱强 669 +邱艳 734 +邱芳 775 +邵伟 593 +邵娜 677 +邵洋 945 +邵涛 959 +邵艳 607 +邵超 750 +邹平 731 804 +邹强 620 +邹桂英 1025 +邹秀兰 1017 +邹艳 703 +郑军 793 845 +郑秀英 738 +郝勇 759 +郝敏 787 +郝明 630 +郝桂英 655 +郝洋 714 +郝涛 898 +郝静 982 +郭娟 680 853 +郭艳 773 +金杰 981 +金洋 812 +金秀兰 719 +钱刚 720 +钱娜 594 +钱敏 954 +钱洋 805 +钱秀英 598 814 +锺勇 767 +锺敏 1008 +锺洋 557 +锺霞 582 +阎军 913 +阎娜 691 +阎平 623 +阎敏 604 1026 +阎艳 882 +陆娟 774 +陆明 760 +陆霞 815 +陆静 564 +陈勇 650 +陈杰 851 +陈洋 625 +陈涛 1007 +陈磊 756 +陈霞 819 +雷丽 850 +雷秀英 543 +雷芳 843 +韩军 947 +韩强 549 +韩敏 895 +韩涛 1019 1033 +韩磊 906 +韩艳 685 +顾军 983 +顾娜 950 +顾敏 735 834 +顾明 900 +顾桂英 924 +顾涛 662 +顾秀英 579 795 997 +顾超 879 +马伟 826 +马刚 847 +马平 832 +马强 700 +马洋 974 +马磊 1012 +马艳 627 +高秀兰 801 +高艳 636 +高超 649 +高霞 634 +魏军 828 +魏刚 699 +魏勇 603 +魏平 698 +魏明 584 +魏杰 831 914 +魏桂英 788 +魏洋 628 +魏艳 996 +魏超 899 +黄强 915 +黄敏 839 919 +黄秀英 1010 +黄芳 748 +黄静 709 +黎刚 1001 +黎娟 922 +黎平 739 +黎明 733 +黎杰 602 +黎磊 694 713 902 +龙伟 875 +龙刚 809 +龙勇 990 +龙平 675 +龚娜 745 +龚平 778 +龚强 948 +龚静 641 + diff --git a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_column_separator.csv b/regression-test/data/load_p0/http_stream/test_http_stream_column_separator.csv similarity index 100% rename from regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_column_separator.csv rename to regression-test/data/load_p0/http_stream/test_http_stream_column_separator.csv diff --git a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_data_model.csv b/regression-test/data/load_p0/http_stream/test_http_stream_data_model.csv similarity index 100% rename from regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_data_model.csv rename to regression-test/data/load_p0/http_stream/test_http_stream_data_model.csv diff --git a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_line_delimiter.csv b/regression-test/data/load_p0/http_stream/test_http_stream_line_delimiter.csv similarity index 100% rename from regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_line_delimiter.csv rename to regression-test/data/load_p0/http_stream/test_http_stream_line_delimiter.csv diff --git a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_multiple_times.csv b/regression-test/data/load_p0/http_stream/test_http_stream_multiple_times.csv similarity index 100% rename from regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_multiple_times.csv rename to regression-test/data/load_p0/http_stream/test_http_stream_multiple_times.csv diff --git a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_orc_case.orc b/regression-test/data/load_p0/http_stream/test_http_stream_orc_case.orc similarity index 100% rename from regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_orc_case.orc rename to regression-test/data/load_p0/http_stream/test_http_stream_orc_case.orc diff --git a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_parquet_case.parquet b/regression-test/data/load_p0/http_stream/test_http_stream_parquet_case.parquet similarity index 100% rename from regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql_parquet_case.parquet rename to regression-test/data/load_p0/http_stream/test_http_stream_parquet_case.parquet diff --git a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql.out b/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql.out deleted file mode 100644 index c5fc17de23f2a45..000000000000000 --- a/regression-test/data/load_p0/stream_load_with_sql/test_stream_load_with_sql.out +++ /dev/null @@ -1,534 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !sql1 -- -10000 aa -10001 bb -10002 cc -10003 dd -10004 ee -10005 ff -10006 gg -10007 hh -10008 ii -10009 jj -10010 kk - --- !sql2 -- -10000 4444 aa 5555555 111.111 3.14 -10001 3333 bb 666 222.222 5.32 -10002 2222 cc 453 333.333 4.321 -10003 1111 dd -3241 444.444 1.34 -10004 -9999 ee 21342 555.555 1.22 -10005 8888 ff 64562 666.666 9.133 -10006 -7777 gg -12313342 777.777 8.1 -10007 6666 hh 314234 888.888 34.124 -10008 -5555 ii 1341 999.999 342.12 -10009 4444 jj -123 111.111 11.22 -10010 -3333 kk 12314 222.222 13.0 - --- !sql3 -- -10000 aa 2017 10 1 -10001 bb 2017 10 2 -10002 cc 2017 10 3 -10003 dd 2017 10 4 -10004 ee 2017 10 5 -10005 ff 2017 10 6 -10006 gg 2017 10 7 -10007 hh 2017 10 8 -10008 ii 2017 10 9 -10009 jj 2017 10 10 -10010 kk 2017 10 11 - --- !sql5 -- -10000 aa 2017-10-01 0 99999 -10001 bb 2017-10-02 0 99999 -10002 cc 2017-10-03 0 99999 -10003 dd 2017-10-04 0 99999 -10004 ee 2017-10-05 0 99999 -10005 ff 2017-10-06 0 99999 -10006 gg 2017-10-07 0 99999 -10007 hh 2017-10-08 0 99999 -10008 ii 2017-10-09 0 99999 -10009 jj 2017-10-10 0 99999 -10010 kk 2017-10-11 0 99999 - --- !sql6 -- -10000 aa 北京 false 11 4444 5555555 41232314 3.14 123.3423 111.111 111.111 2017-10-01 2017-10-01 2017-10-01T06:00 2017-10-01T06:00 -10001 bb 北京 false 22 3333 666 2768658 5.32 123111.3242 222.222 222.222 2017-10-02 2017-10-02 2017-10-02T07:00 2017-10-02T07:00 -10002 cc 北京 true 33 2222 453 5463456 4.321 11111.23423 333.333 333.333 2017-10-03 2017-10-03 2017-10-03T17:05:45 2017-10-03T17:05:45 -10003 dd 上海 true 44 1111 -3241 -45235 1.34 54626.324 444.444 444.444 2017-10-04 2017-10-04 2017-10-04T12:59:12 2017-10-04T12:59:12 -10004 ee 成都 false 55 -9999 21342 4513456 1.22 111.33 555.555 555.555 2017-10-05 2017-10-05 2017-10-05T11:20 2017-10-05T11:20 -10005 ff 西安 false 66 8888 64562 4356 9.133 23423.45 666.666 666.666 2017-10-06 2017-10-06 2017-10-06T12:00:15 2017-10-06T12:00:15 -10006 gg 深圳 true 77 -7777 -12313342 34534 8.1 12.0 777.777 777.777 2017-10-07 2017-10-07 2017-10-07T13:20:22 2017-10-07T13:20:22 -10007 hh 杭州 false 88 6666 314234 43535356 34.124 324.0 888.888 888.888 2017-10-08 2017-10-08 2017-10-08T14:58:10 2017-10-08T14:58:10 -10008 ii 上海 true 99 -5555 1341 23434534 342.12 34234.1 999.999 999.999 2017-10-09 2017-10-09 \N \N -10009 jj 南京 false 11 4444 -123 53623567 11.22 324.33 111.111 111.111 2017-10-10 2017-10-10 2017-10-10T16:25:42 2017-10-10T16:25:42 -10010 kk 成都 false 22 -3333 12314 674567 13.0 45464.435 222.222 222.222 2017-10-11 2017-10-11 2017-10-11T17:22:24 2017-10-11T17:22:24 - --- !sql7 -- -10000 aa 西安 22 0 1234567 陕西西安 2016-02-21T07:05:01 -10000 aa 北京 21 0 1234567 北京 2017-03-11T06:01:02 -10001 bb 上海 20 1 1234567 上海 2012-05-22T12:59:12 -10001 bb 天津 33 1 1234567 天津 2019-01-11T17:05:45 -10002 bb 上海 20 1 1234567 上海 2013-06-02T12:59:12 -10003 cc 广州 32 0 1234567 广东广州 2015-08-12T11:25 -10003 cc 广州 32 0 1234567 广东广州 2014-07-02T11:20 -10004 dd 杭州 47 0 1234567 浙江杭州 2017-11-23T13:26:22 -10004 dd 深圳 33 1 1234567 广东深圳 2016-12-01T14:04:15 -10005 dd 深圳 19 0 1234567 广东深圳 2018-10-03T12:27:22 -10005 ee 成都 21 1 1234567 四川成都 2019-09-03T11:24:22 - --- !sql8 -- -10000 aa 西安 22 0 1234567 陕西西安 2016-02-21T07:05:01 -10001 bb 上海 20 1 1234567 上海 2012-05-22T12:59:12 -10002 bb 上海 20 1 1234567 上海 2013-06-02T12:59:12 -10003 cc 广州 32 0 1234567 广东广州 2015-08-12T11:25 -10004 dd 杭州 47 0 1234567 浙江杭州 2017-11-23T13:26:22 -10005 dd 深圳 19 0 1234567 广东深圳 2018-10-03T12:27:22 -10005 ee 成都 21 1 1234567 四川成都 2019-09-03T11:24:22 - --- !sql9 -- -10000 aa 西安 22 0 1234567 陕西西安 2016-02-21T07:05:01 -10001 bb 上海 20 1 1234567 上海 2012-05-22T12:59:12 -10002 bb 上海 20 1 1234567 上海 2013-06-02T12:59:12 -10003 cc 广州 32 0 1234567 广东广州 2015-08-12T11:25 -10004 dd 杭州 47 0 1234567 浙江杭州 2017-11-23T13:26:22 -10005 dd 深圳 19 0 1234567 广东深圳 2018-10-03T12:27:22 -10005 ee 成都 21 1 1234567 四川成都 2019-09-03T11:24:22 - --- !sql10 -- -1500 - --- !sql11 -- -10000 aa -10001 bb -10002 cc -10003 dd -10004 ee -10005 ff -10006 gg -10007 hh -10008 ii -10009 jj -10010 kk - --- !sql12 -- -10000 aa -10001 bb -10002 cc -10003 dd -10004 ee -10005 ff -10006 gg -10007 hh -10008 ii -10009 jj -10010 kk - --- !sql13 -- -4632802378222380466 1 null Acoper «labilir mitsubishi в Липецке на Batak 1 2014-03-23T16:23:07 2014-03-23 27528801 2723656327 7fc6e55c74e3029834ae16bf67d8cb9f4f 15887 2224216775939367616 0 56 4 http://rutube.ru/patianu http://love/v012/04 footki.yandex.ru m.fotoalmeta 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3238011 -1 0 0 0 1136 555 117 2014-03-23T06:04:09 5 1 19832 0 utf-8 401 0 0 0 0 1018986580 0 0 0 0 0 5 2014-03-24T08:02:03 16 2 2 0 0 [] 2529497135 30d877a66ec8945d1e4d056e51e3de35 18372 -1 2 wi a5b3 0 -1 4333 1332 747 2472 0 0 7395 285 -1 -1 -1 16115 -1 -1 0 0 07d21f 0 [] 0 307141980878902893 338497349483004835 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 47 0 -4716566754160586468 1 Тимошенковске на F1news: Новгородажа легководителей и ее европы - Странспорт 1 2014-03-23T10:52:43 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.139517&quickpantinoyleri http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 943 296 2014-03-23T22:23:20 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 278406132 0 0 0 0 0 E 2014-03-23T20:23:36 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 0 0 1 2 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 7982945165681148628 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -4755557211339260600 1 PwC 1 2014-03-23T16:25:12 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/index?appkey=&m[]=6&frommail.yandsearch?text=никол в про и невиннисаж http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T05:54:31 0 0 0 0 utf-8 401 0 0 0 0 346099590 0 0 0 0 0 E 2014-03-24T08:01:40 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 59 78 0 0 2251 0 -1 -1 -1 3362 798 -1 0 0 07d21f 0 [1555356] 0 6331778101536488133 6157439411447496521 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -4770046483886760822 1 Mercedes-Benz e-клас. Можно купить 1 2014-03-23T16:26:50 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/weatherstyle/29/127737064&domain=hurriyet.com/volgogram/22/10/Default-tistings=535674/atturkiye http://mynet.com/mate-guzellileridetay/80508/bio/video?q=породажа хабаров с технологических mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T06:02:18 0 0 0 0 utf-8 401 0 0 0 0 782597291 0 0 0 0 0 E 2014-03-24T08:03:20 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 96 115 252 0 0 6165 306 -1 -1 -1 8173 1287 -1 0 0 07d21f 0 [1555356] 0 8744694472066974558 13107909194127435888 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -4831817364509130205 0 Сериалы 1 2014-03-23T20:34:29 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 http://yandsearch.php?id=209516817950 tranamaschools top.rusfisha.msu 0 0 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 -1 0 0 0 1 1 117 2014-03-23T06:05:12 0 0 0 0 utf-8 401 0 0 0 0 6030782 0 0 0 0 0 E 2014-03-24T14:17:35 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 -1 0 0 3018 0 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11033243704253757784 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -4835752945340096735 0 Продать, предложений — AVITO.ru. Часть бесплатно онлайн Фильма «T+2» (286): 1 2014-03-23T12:16:06 2014-03-23 1785 44808656 0442e62a5c5cb134d699fbae72bc9c588b 7 454237030118460538 1 3 3 http://91111478334250732/?page=20&i=1&parenk=&changed http://yandsearch;web;;%2Fwwwww.silverdi-kading/site_id=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJpvc9LIo8yqpc1a7i7QPYfRisPDxcOAMMjcyNjQ3MTIxfcSMTQ0Y2U4OWQ5WWVFUnFSWhVTUNjVmN2N2 coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1297 555 29 8 0 0. 0 0 17 73d7 1 1 0 0 1384917 3 2 0 0 1366 1019 209 2014-03-23T20:26:57 3 0 58982 0 utf-8 401 0 0 0 5636140508787295899 929361662 0 0 0 0 0 E 2014-03-23T22:24:51 55 2 2 675 0 [4,5,80,82,2193,285,265,76,14,10,112,3,9] 46871203 be72ce3c669bd45ee99b8e038d989526 -1 -1 -1 nD Tp 0 -1 0 0 19 11 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10605622915436919128 11933878103501891696 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -4837660920166529023 1 Сбербанк ОнЛ@йн 1 2014-03-23T13:27:36 2014-03-23 1785 280750947 99ebef8641c09581865c6e5c72b33f05895a 19973 322556318799166472 1 56 2 http://9111147842,0.01178.13950555&lr=66&img_url http://kelebekgaleri.net/besplayer&join.ua/rezerval=1&bodystyle.com/8-marter97.org/remont/innopoisk.ru/news.mail.ru/video/search.xml?categoryId=5000&year][max]=1002-fh-467-pd-1&rpt=simages.yandex/uchita.ru/view/133-ukrainy-duoi tronika.html?l=prod.dp.ua/?page/61307-fh-467-pd-1&p=17&uinfo.aspx?wt_rp=27&hid=91651068.html&lr=213&text=search?lr=2177467203159_5500000191&text=бекление&filmId=P90GJOSRzJSNWM%3DfnSZA%3DfeSfgSMQ%3DfoSfpSYmJTNFRFJzVjdkRNQ2VWRoUFNuTDJPZnJ0U3ZnhaMlJ1bEhoVTE2WFJRTzJ1bUl9HP3J5ftXDKDyocZuDO7C4iTUDb8chOvsX2R1YmU9ecE4j44QTjhNRDRpRE40ZFdElB&b64e=2&sign=89846355c1c50b406146965/36733f0427100&sfrom=592&m[]=5431ecedes_businiz-view957448172/?from=yandex.ru;yandex.ru/a-sezon coccoc.com sprashivai.ru 0 0 [5,353,3121,11503,11630] [119,3418,2455,3036] [18] [28] 1339 555 57 8 0 0. 0 0 29 73d7 1 1 0 0 1412357 3 3 ходячие раскрасны 0 0 1997 891 209 2014-03-23T20:32:22 0 0 0 0 utf-8 330 0 0 0 8168139702751979718 408832541 0 0 0 0 0 E 2014-03-23T23:57:58 16 2 2 2 0 [9,3274,1,14] 421494715 f20bb2faf420abe95559d6bcfe354d05 -1 -1 -1 nD a5b3 0 -1 0 35 58 56 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 18126425332249278808 14950347525130479728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -4850926408483995792 0 Современа - Футбольших 1 2014-03-23T16:23:19 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1_844_128338&text=ниссат б3 2.0 скачать http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vcG5ld3MtUUov2KB1GZjamJIeW1nQ3NGMkVMNEZBa3hpZ2VzLzI3L25ld3MvMjgw games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:46:01 0 0 0 0 utf-8 401 0 0 0 0 583663939 0 0 0 0 0 E 2014-03-24T07:52:16 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11875920750199978328 4452926215554207674 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 107 0 -4926263996196211822 1 В пробегом - катеристочник 1 2014-03-23T16:21:04 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T05:53:35 0 0 0 0 utf-8 401 0 0 0 0 1056965251 0 0 0 0 0 E 2014-03-24T07:57:58 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 21 49 40 104 0 526 0 -1 -1 -1 1986 1645 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -5017198962525199154 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:57:36 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:36 0 0 0 0 utf-8 401 0 0 0 0 658682892 0 0 0 0 0 c 2014-03-24T07:38:20 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 17131 -1 1 wi a5b3 0 -1 0 0 1 3 0 0 241 0 -1 -1 -1 410 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -5105151702377877258 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:12:37 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:43:14 0 0 0 0 utf-8 330 0 0 0 0 419812360 0 0 0 0 0 5 2014-03-23T11:13:03 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 2 nD Tp 0 -1 0 0 287 327 12 0 529 1480 -1 -1 -1 1501 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 750 0 -5138975406131988633 1 Сбербанк ОнЛ@йн 1 2014-03-23T13:28:24 2014-03-23 1785 280750947 99ebef8641c09581865c6e5c72b33f05895a 19973 322556318799166472 1 56 2 http://9111147842,0.01178.13950555&lr=66&img_url http://kelebekgaleri.net/besplayer&join.ua/rezerval=1&bodystyle.com/8-marter97.org/remont/innopoisk.ru/news.mail.ru/video/search.xml?categoryId=5000&year][max]=1002-fh-467-pd-1&rpt=simages.yandex/uchita.ru/view/133-ukrainy-duoi tronika.html?l=prod.dp.ua/?page/61307-fh-467-pd-1&p=17&uinfo.aspx?wt_rp=27&hid=91651068.html&lr=213&text=search?lr=2177467203159_5500000191&text=бекление&filmId=P90GJOSRzJSNWM%3DfnSZA%3DfeSfgSMQ%3DfoSfpSYmJTNFRFJzVjdkRNQ2VWRoUFNuTDJPZnJ0U3ZnhaMlJ1bEhoVTE2WFJRTzJ1bUl9HP3J5ftXDKDyocZuDO7C4iTUDb8chOvsX2R1YmU9ecE4j44QTjhNRDRpRE40ZFdElB&b64e=2&sign=89846355c1c50b406146965/36733f0427100&sfrom=592&m[]=5431ecedes_businiz-view957498&answer=&channe_2010_hand.ru/chase/aleksey510376 coccoc.com sprashivai.ru 0 0 [5,353,3121,11503,11630] [119,3418,2455,3036] [18] [28] 1339 555 57 8 0 0. 0 0 29 73d7 1 1 0 0 1412357 3 3 ходячие раскрасны 0 0 1997 891 209 2014-03-23T20:33:18 0 0 0 0 utf-8 330 0 0 0 8168139702751979718 668154955 0 0 0 0 0 E 2014-03-23T23:58:33 16 1 2 2 0 [9,3274,1,14] 421494715 f20bb2faf420abe95559d6bcfe354d05 -1 -1 -1 nD a5b3 0 -1 0 0 14 2 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10100085969557141848 14950347525130479728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5291067970604774260 1 Максай игру роботы, купить в интернет-магазин спецпредства воорешебниках - вакансии лет — Сеть легководставито 1 2014-03-23T05:07:16 2014-03-23 15665476 1087886866 269c5c27e4946a667f78888c9f75b35c6e6e 49701 334328984627246488 1 56 5 http://mail=1&search?text=гугл перевод старый сайт с шоп&strirodam_spanie-podkRPaXptc3hrVndyR2JaRFUxWTJ4aFVG http://hood_search&event_id%3D84840.2402105-50&uuid=&status=0;orderevery-club.me/berkusurulu-bolum/2522&language=2014&page2/?go market.yandex.ru.msn.com pegasha-pogoda 0 0 [353,3121,11503,11630] [] [] [28] 1174 809 29 8 0 0.77 0 0 15 73d7 1 1 0 0 3560775 3 4 погода на рассаж 7 0 0 1962 676 157 2014-03-23T06:41:45 0 0 0 0 utf-8 401 0 0 0 0 419939021 0 0 0 0 0 E 2014-03-23T00:07:02 55 2 3 32 103 [2193,265,82,6,95,288,14,100,3275,72,3,76] 1477819126 1be50e4c7673a88c47cace3e47478733 -1 -1 -1 nD Tp 0 -1 0 0 3 30 61 0 25 -1 -1 -1 -1 -1 52 -1 0 0 07d21f 0 [] 0 8851578361410429238 10331158898916649072 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5305634367927610522 0 Современа - Футбольших 1 2014-03-23T16:23:48 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=19554212079d1b7cbb8a6a2.jpeg%3DfeSfgSYmI%3DfdSNTU2feSMzAw http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb21f3fc3aaa24dc63a21xZGVqSE1GcFQxUkJN&b64e=2 games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:46:25 0 0 0 0 utf-8 401 0 0 0 0 167969307 0 0 0 0 0 E 2014-03-24T07:52:45 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 3480781366221101891 1661165035492520768 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 127 0 -5309644206721379498 1 Письмоградской 1 2014-03-23T20:55:44 2014-03-23 27941559 1767624985 a7da21818799159fb389b18d4373596b 225 2721568806677236614 1 56 3 http://patia_spark ns7&search?clid=9582 http://hood_search&event=list/casing-cap tvmir.nu pegasha-pogoda 0 0 [353,3121,11503,11630] [5,7,73,400] [18] [28] 1174 809 29 8 0 0. 0 0 17 73d7 1 1 0 0 2179614 3 4 герб марта спицам терми 0 0 1731 391 117 2014-03-23T05:14:07 0 0 0 0 utf-8 401 0 0 0 0 777203605 0 0 0 0 0 E 2014-03-23T13:07:48 0 0 0 0 0 [] 1852761877 b016f1c7c4de86f1c384f8d63570a742 -1 -1 -1 nD Tp 0 -1 0 0 460 199 31 0 1530 0 -1 -1 -1 2846 -1 -1 0 0 07d21f 0 [] 0 15279646984699132248 8418678203875343898 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5420983836600446463 1 Продажа Toyota / Результиворк! 1 2014-03-23T03:35:14 2014-03-23 1785 1950378921 c2602efb1311636cf875df404f3d6529 51 3206084085559802010 1 56 4 http://9111147854&change_key=506d9e3dfbd268e6b6630e58&domain=hurriyet http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ukrayila-yenisafak.com%2Ftitle,pos,p0,source,web&text=кредиторожный краска длинны онлайн&uuid=0&count=135238478226d6ff8ed00dMU7UvmIoQmkzfPmqekNCSExDRVoxZEcxcGJsN3kxUUxhalleonary_to=2014 sony_price_usd[1]=&proigry-dlya-suka.ru/sankt-peterburgskaya_oblast_volume[2]=&color_id=0&year[1]=&private/en/sessuary?p=90&q=1395592f72162c019715ade9befae9562669,"first_name":"Альберт&clid=195545452&lr=109aa7f2a8bc3d75c95724332 coccoc.com tanks.co.il 0 0 [] [119,3418,2455,3036] [18] [] 1297 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 906571 3 3 торта самарекс по оптом из неполных года выхода скалы фотошоп 0 0 1366 593 262 2014-03-23T16:44:09 5 1 14470 0 utf-8 401 0 0 0 4808217662922694360 7065527 0 0 0 0 0 E 2014-03-23T16:14:51 0 0 0 0 0 [] 2137570165 46a233a0e33f11c4fee726c7303f44d5 -1 -1 -1 nD a5b3 0 -1 1 173 292 430 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8411534898364058195 7194548681272151755 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5520081962326584152 0 Ремком крае национны [AVC], VA, Любой в Москве крымская историгина домашними рукодекс: нашлось 3 млн ответов 1 2014-03-23T09:45:17 2014-03-23 1785 1242723321 2684553455bcb892472422768c7b4b2f 13437 3074483810024357617 1 56 2 http://9111147842,0.01178.13956069c61cc8Apdghvwm6ThW9Y4QOdYgU_GUIzigLM8W6g0QMWtNUXI&where=all&filter coccoc.com 0 0 [] [119,3418,2455,3036] [18] [] 1622 1583 57 10 6 0.70 0 0 14 RT 1 1 0 0 3039699 0 0 0 0 2011 726 296 2014-03-23T18:39:54 5 1 19832 0 utf-8 330 0 0 0 8207959345338106813 992900079 0 0 0 0 0 E 2014-03-23T20:03:28 22 0 3 13732 6 [6,219,1549,7,3003,501,999,18,1,36,25] 2143328713 fcb87cf6e44fbb0e1d8456015c663d3786 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4127982339363240333 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 96 0 -5559361293957300414 0 Женская библи (сериал "Фиксика 1 2014-03-23T18:17:33 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/forums.kuban.ru/volgogradmir http://yandex.net/legkovye-avtorii/118534731400928 yandex.ru tury.ru.msn 0 0 [85,350,1212,3849,8358] [106,323,1092,3676,13188] [18] [28] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 -1 0 0 0 592 503 117 2014-03-23T19:19:06 0 0 0 0 utf-8 401 0 0 0 0 647923622 0 0 0 0 0 E 2014-03-23T12:26:14 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 1166 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 884992008826344665 4724303053209794720 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -5619100536162139501 0 QIP.RU - Гайд Dota Construment Steel, Красности стильни госдеп США Джинс | Вакансии. Одежда деловек пау газета.Ru — смотреть лительность ерально и в Новости Украина, фото - View Film izle Çizmet Kay - VAZ, Gereketim Dinlementique – сайте AUTO.RIA - Фишер Успевай 1 2014-03-23T12:34:12 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://9111147854&change_key=506d9e3dfbd268e6b6630e58&domain http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/view_intext=как в москва режим умом заводоровать онлайн бесплатно в любимому языку 4 класс порно встрашный сайт длинновая папины мамедостовск-орловского ли реимущественны и холодилер шеференков&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XsltfjMh8My-4Mp2Kzonmlr9Oj-tkmzK1-_9gEp7c156bcf985&text=sony?p=11&cbv=r2014/03/19/bigparams/174780954.24549/2679&page10/?&p=36&uinfo/index.ru;yandex.ru/krasnodar.org&from=yandex.ru/viewly&kinda-raduga/arams.com/haber/10/155-02-20053.4678203611557414e2725-derby_day=2014&clid=11985&lan=ru&mc=4.804252639b7ba5981421beb&ui=uv3lovAaf3sLhCa43t3Qfk-sdelaya/sobaki/?page/dl/секс поручную под люди&lr=213&text=гарнир__Абсолютное подтяжки&lr=108444071-fiksiki-6-chere=all&filter_id=13451&lr=8&text=диета авито блей&lr=66&bpp=7&i=1&when=2009%26nord%3D%26extras[14]=0&extras[27]=29066462.0819084f155715641255491/dating/used/sale%26d%3D26%238242_259066068085843223.html?ia=M&lf=M&af=18&pw=1&target=search.phtml?pid=89§ion1[3]=&searchastnik.ru/my/hit_list/work.ua/clck/jsredirected=13859&text-albinoy-kutladikavkaz/makler.ru/aclk?sa=l&ai=C9QzwyeUQVCiFZB79rQYLQ1GPgN7Qi82fGpeBl0LLWFkQlRDAlQjUlRDAlQjklMjQ5dWFWOW9iM1I1WVhKdld5NDV1MOXXox_OxcFDY-uop-thersoy-tonkiimir_snovachale-secim.html#/battlemena.avito.ru/cars.auto.drom.ua/manage=eyAidW5yZWFsdHkvMjAxNDAzMTgiO3M6MjoiaWQiO3M6ODoiMjAxNzE0NDk0NzUva$84NzI2Mjk0NzttYAw&usg=AFQjCNGheBpruD0hte0FyaUqCNDk3DqQvygvalcenkin/offers.xml?category_id=199555453&banner_pf[HM_S106Wpb7rjrgxoAGGkuc2svdGhlYXRyemst=my.mos.ru/yandex.ru/companoritecomm coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 орхидея в мирандашом ценарисы гузеевой мото рисунок и простройщика фото с поздравлений телеканка фото 2014 год собласти ужгород 0 0 1389 884 157 2014-03-23T22:34:54 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 846684171 0 0 0 0 0 E 2014-03-23T22:48:13 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10287839608055806296 17253681420892301424 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 89 0 -5711297485370126517 1 Голос видео 1 2014-03-23T03:51:10 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/triket/4746734017811fc051dd0e57 http://yandex.ru;yandex.ru/?area=ShowForum.ru/look/event cdo.e1.ru.plays олины.рф 0 0 [] [] [] [] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 -1 0 0 0 1460 549 574 2014-03-24T03:26:32 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 585440942 0 0 0 0 0 5 2014-03-23T02:24:58 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 55446 -1 5 wi a5b3 0 -1 0 0 454 3 0 0 78 37 -1 -1 -1 386 486 -1 0 0 07d21f 0 [] 0 18236259283221887320 16190469452247787632 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -5733212747584636203 1 Как лучшие руках, часа в Красноярска обновлетворящих » Купить качесть по г. Москве на AVITO.ru (Недвижимость на AVITO.ru. Стулья салоны в Владимир 1 2014-03-23T13:20:49 2014-03-23 1785 2501928799 4a31e16945f0d802d9dc0705979c4672 239 4517116963127554624 1 79 4 http://9111147811.html?v=1393_0_251311310 http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 2081850 3 2 0 0 1997 593 441 2014-03-23T03:13:35 0 0 0 0 utf-8 330 0 0 0 7146369058166890546 793191128 0 0 0 0 0 E 2014-03-23T23:56:34 16 1 3 12333 0 [14,1,6,501,119,72,9,510,3540,925,22,11] 2976974125 a3831324980206d1415afea636cc7635 -1 -1 -1 nD a5b3 0 -1 92 125 121 201 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 12557946248999135344 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5772344969531870975 1 «Сбербайджане, вакансий и джемпер цены найдено в Яндекс.Работа для занятия числе за 450 000 рублей ПК | Купить Ladie | News.ru - Новости • Продаже к Eurospor-Fenerbahçe Speed: Resmi intine траханик, - Politikam alın mustafa Keyfiniti JX внедопуска | mp3, видео – частных фото эротив Украинская Компаний в Московским 1 2014-03-23T03:16:04 2014-03-23 1529 374391783 e3b7a9be7902b95c5cf957839f5c66920e71 154 1688129825915497537 1 56 3 http://realty.dmir.ru/yandex.kz/Tsvet-risunki http://apps/docview/Kvasi.html&ei=cL5HMPDC31TNXJ0MHZlWHNVctY2ozU0EzNUJVddxVjBmc1ld9IX1iEAbgEDgu kolesa.kz forum.print.ru.msn.com.travel 0 0 [2672,3170] [5,7,96,420,477] [] [28] 1846 849 29 8 0 0. 0 0 24 73d7 1 1 0 0 1072276 3 12 поезда молдованной 0 0 785 832 296 2014-03-23T00:08:04 0 0 0 0 utf-8 401 0 0 0 6192436962476724500 979256876 0 0 0 0 0 E 2014-03-23T20:34:42 0 0 0 0 0 [] 327000091 3fe367918369045361fae91e8d77b859 -1 -1 -1 nD Tp 0 -1 49 0 642 1 1637 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6855930052452742827 6759968051075183589 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5869363305929907643 1 GALAXY S4 milyon sonuç bulundu! | Независтная можешь не с коттеджи - Новости: чемпион» 1 2014-03-23T04:15:40 2014-03-23 15665476 3911193819 75af3860b05c745c5c5c27b30f6ab2ed546589 196 527703306321131114 1 56 4 http://lima-lgot.php?t=7684-3c1d3ca8bf948f9add4993509/6257644%2F04725054%2F661%2F52499 http://yandex.ru/yandsearch?lr=213&oprnd=9161312&search?text=уход мерседес аста сараты&lr=2&tag=studen-twittemplatyana-ne-pistoryid=84643938c0395/album_id=19720.com/fuatavni--4292-5-0-271-zazda_3_/?curre=0&shv=r20140316862,bs.1,d.bGE&cad=rjt images.yandex.ru.msn.com.ua yandex.ua 0 0 [353,3121,6479,8856,11503,13770,14847,11630] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 1769215 3 1 в россии отности 0 0 1789 602 157 2014-03-23T05:44:47 0 0 0 0 utf-8 401 0 0 0 6176299642330297673 798409806 0 0 0 0 0 E 2014-03-23T23:18:58 16 2 3 11502 7 [6,2,119,95,2193,5,272,109,3275,1,28,70,14] 2533100459 6bc04605521abb96d790e6be4375ac1f -1 -1 -1 nD a5b3 0 -1 1 55 213 0 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2391497380856937231 16719441483451263088 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5872091609206767166 1 В пробегом - катеристочник 1 2014-03-23T16:17:06 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:48:18 0 0 0 0 utf-8 401 0 0 0 0 1035856160 0 0 0 0 0 E 2014-03-24T07:53:45 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 22 31 62 42 0 0 1327 0 -1 -1 -1 1381 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 252 0 -5894101013807685227 1 Kia Royal manga online, MediaPortaya barsu — стройки - Страну г. Модная фанфик пришли с животных ведущий Волк 2 млн ответов посмотреть 1 2014-03-23T11:16:53 2014-03-23 1785 2750008691 d27e873642fdcc142fd52f628470ba33 241 7406071195777269826 1 156 122 http://911114786987.html#/battle-en47.html?field/rd.dometrov http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/sitelink_id=0&year[2]=&extras[23]=0&search.xml?&model=false&showtopic.php?w=728&text=пдд 20140321%2F17-032-GWAY&text=личный связной&relocatid=207&q=samsung&q_page=0&client_id=200000943.aspx?DocID=240&sTo=0&l10n=ru&mc=4.12485408077879-PF=214240d26008706,2796&lr=35&win=20000&pmin=100&text=&etext=&search?lr=213&lr=960&t=o&sz=b&s_c=3159_37612330ea&keyno=0&l10n=ru&mc=5.4044324_40273&ga_vid=1&ved=0CIsBEIQcMCM&img_url=http://yandsearch?cl4url=aHR0cDovL21vc1ZOSW5AY9mSDAb-8Ep9GVzJ6R0xsNkVWRGT0D8ZTzZITkpPS2hHd058b3f7652/page=links,pos,p2,source,web&text=бэнтэн 10 коп 2014 иван недвижимость дому&rpt=nnetter-1-seriya-na-rabota.allbusinema coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 13 8w 1 1 0 0 2723983 3 3 натяжные моряк диски для короволос 0 0 1997 547 157 2014-03-23T21:33:37 0 0 0 0 utf-8 401 0 0 0 8546017116003926276 479894081 0 0 0 0 0 E 2014-03-23T21:51:09 22 2 2 0 0 [] 2771758996 1c0ec512cf36858257e85c74d2ac7654a2 -1 -1 -1 nD a5b3 0 -1 1 46 79 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 7713159565439621005 15167358802972043376 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -5920271052064151811 1 Письма сание 1 2014-03-23T16:27:56 2014-03-23 12461549 4071235651 fab2fa83e0438d1759eecbe52a5c5c2861 9580 532344165497878132 0 56 4 http://yandex.ru/c/11393489.0744 http://go.mail.ru&js=1&ref mail.yandex.ru yandex.ru 0 0 [353] [3666] [903] [28] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 2608898 3 2 0 0 1136 593 117 2014-03-23T05:51:52 3 0 51135 0 utf-8 401 0 0 0 4969423154180479309 317525830 0 0 0 0 0 E 2014-03-24T07:58:16 55 2 2 0 0 [] 2774799269 5998146c305c74c6c4bb7efb9b8586f333 -1 -1 -1 wi a5b3 0 -1 869 2 1010 390 0 0 578 0 -1 -1 -1 2739 2266 -1 0 0 07d21f 0 [] 0 4313994922287858055 3771131554017970634 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5969822703110278505 1 Hyundai Accord в Новостелефоны 1 2014-03-23T05:23:01 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/for/tvorcher.taka.com http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T05:29:49 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 1073493585 0 0 0 0 0 E 2014-03-23T03:47:26 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 3 wi a5b3 0 -1 0 0 770 6 382 446 423 162 -1 -1 -1 1798 1877 -1 0 0 07d21f 0 [] 0 4313994922287858055 11715451638231263344 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 8 0 -6052478491822619321 1 Opel / Результик и быть суточных, серты на AllBoxing.net Email.Ru: Cemilie screed for movie Trans 1 2014-03-23T10:52:53 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://911114788/34113800297462973291/140/Default?prodau_dvukhkolaeva http://fotogrammyeli_i_ukraine-sims3packcpm.php?topic.php%3Ftarget coccoc.com m.mylove.mail.ru 0 0 [116,2586,3324,2253] [119,3418,2455,3036] [18] [28] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 -1 0 0 0 656 943 296 2014-03-23T22:23:33 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 434567472 0 0 0 0 0 E 2014-03-23T20:23:45 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 0 0 483 200 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 4982207522207701191 14629325741541222512 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -6133511635323783592 0 Доска без поступавная празделия положилище 1 2014-03-23T11:10:20 2014-03-23 1946 1936436644 2a53729da8c204bd5c6270cce24a479a4f 196 11626159163821345859 1 223 42 http://avito.ru/for/spravoslavl.upravili.ru yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 317 763 57 0 0 0 0 4 73d7 1 1 1 1 S820_ROW 647650 0 0 0 0 592 1639 157 2014-03-23T03:51:31 0 0 0 0 utf-8 401 0 0 0 0 729839554 0 0 0 0 0 E 2014-03-23T03:36:02 0 0 0 0 0 [] 1982208793 be2c1adef2c444e4237e7e2837889d34 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 2529151358065175971 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 0 0 -6144733558785969432 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:13:39 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:44:27 0 0 0 0 utf-8 330 0 0 0 0 358446447 0 0 0 0 0 5 2014-03-23T11:14:24 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 -1 0 0 11 105 28 0 353 23 -1 -1 -1 811 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 866 0 -6164438624456261208 0 Маринговых домашних услуги комнатную руками смотреть онлайн бесплатно 1 2014-03-23T21:12:12 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogo_baskalabs.html tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 0 412 567 296 2014-03-23T06:42:52 0 0 0 0 utf-8 330 0 0 0 0 529683494 0 0 0 0 0 E 2014-03-23T11:12:43 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 1 nD Tp 0 -1 47 0 251 0 7 0 168 12 -1 -1 -1 894 -1 -1 0 0 07d21f 0 [2170618] 0 15284527577228392792 2665691200259067948 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 793 0 -6209194776693341710 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:56:49 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 1 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:02 0 0 0 0 utf-8 401 0 0 0 0 968302720 0 0 0 0 0 E 2014-03-24T07:37:21 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 43140 -1 1 wi a5b3 0 -1 1 27 38 3 0 0 1791 0 -1 -1 -1 1897 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6223347640367363758 0 Задач на AUTO.RU: Поиск туры в Липницкой Радиоактивы -2015 год геев. Учены на SEfan.RU 17.03.2013 смотреть легковой закансии. Продажа легальном из ЕС бегства, запчастей. MuzMix.com - AVITO.ru — страница с России (56): Яндекс.Музыка для помогите популярности, динами: 4490210 1 2014-03-23T13:47:05 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://911114785-fw-112889455876.jpg","photostrator http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/clck/jsredirectadvert_month=2010&year=2002324.vkrug-orgtehnicspirit.ru/imgrefurl=www-898-ws40-10058/links,pos,p18,source,web&text=медованнал стс оформеропор"}]}&ref[0][paratovsk.ru%2F&lr=16&ct=adprev_fmts=728&uuid=&price_val1=&vol25/2/1/2/3/soru-kirdi/6567/47210%2F17mar1788&oprnd=49792&group-sedan/used/kampiyonlain-perednie-voennym_retro-cc.ru/yandex.ua/lenta.ru/catalogue/38939f0a656b977433296_33430-fw-1366&search;web;;%2Fweb%2Fitem%2Ftitle,pos,p16,source,web&text=недостановосибирск картинки&clid=157&oprnd=8782fef9e35b96627b4a237a8548.1&__utmx=-&__utmz=2111-iz-2755&lr=1036383bab44e22e6841483&sign=95&c=1&group_rul/cev-ustry=11882610&type=2&sign=cd73ec8b341b21cec73/23681912557914~EQ~sel~sel~x1561845/?i=1&w=экопольную информа между черезидения&uuid=&pricesmin=300&wp=1&searchText-seye-rovench-bank.ru/fore/1728&uuid=&subscribe_id=0&extras[2]=13658857.6753039669e18799961c\\\\%2F537475t2JFdkhSN1ZnNhdkx2M0NzE2VUl2WjNYanN6ak5ZVNMdzJSUWcwcDg5aUctaXhiZTVXTkU4V2VKd0toY0FrWkdiRkpLWEVCYo9vUg&usg=AFQjCNHCw82bldXYklOZnJyMDNVbVk&b64e=2&sign=43640d834a72f16b5872 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 орхидея головых отправоохожденных перечественниями 0 0 1389 884 157 2014-03-23T23:59:59 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 34846056 0 0 0 0 0 E 2014-03-23T00:15:22 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13223569481380149592 15473670750921746544 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 115 0 -6226406851137964111 0 Сериалы 1 2014-03-23T20:34:14 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 http://yandsearch.php?id=209516817950 tranamaschools top.rusfisha.msu 0 2 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 -1 0 0 0 1 1 117 2014-03-23T06:04:49 0 0 0 0 utf-8 401 0 0 0 0 122853544 0 0 0 0 0 E 2014-03-24T14:17:24 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 -1 0 0 935 0 0 0 285 0 -1 -1 -1 1165 -1 -1 0 0 07d21f 0 [] 0 11033243704253757784 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6229935597032769119 1 Серия 9 - 2013. Дети@Mail.Ru — социалист 1 2014-03-23T16:22:58 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://yandsearch?lr=35&rpt=simage&lr=66&text=розаврилова http://google.com/games.mail.ru ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 3 2 0 0 1994 561 117 2014-03-23T06:46:48 0 0 0 0 utf-8 401 0 0 0 9007862211105217806 845095662 0 0 0 0 0 E 2014-03-23T23:46:59 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 0 0 1 1 0 0 562 -1 -1 -1 -1 -1 303 -1 0 0 07d21f 0 [] 0 1930741556500475038 9207967286803898539 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -6269707891471564999 0 Путешества@Mail.Ru / Мои сообы сказка 1 2014-03-23T21:11:07 2014-03-23 16862487 1755644572 945af199eed21f3f249577b14f55534b 8363 309245865193991600 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carry_2013.html&forInitQuery=PER_OFFERS_SRCH_MAIN][]=&a1=&a2=30004989614 docviewer.yandex 0 0 [] [443,333,5760] [18] [] 0 0 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 2903448 0 0 0 0 0 0 117 2014-03-23T04:52:19 0 0 0 0 utf-8 330 0 0 0 9110769067793080479 287375675 0 0 0 0 0 E 2014-03-23T13:17:31 16 2 2 0 0 [] 1225767431 2672ee20d87b5c666121145dd76b3644cc 50936 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7914288637035474585 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 61 0 -6310212742328505656 0 Современа - Футбольших 1 2014-03-23T16:22:30 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1955498279&lr=2222_internatik.com.tr&callback http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20#.Uykh-uslug games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:01 0 0 0 0 utf-8 401 0 0 0 0 826263063 0 0 0 0 0 E 2014-03-24T07:51:18 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2316055356621577660 1824019458964194192 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 65 0 -6346147178174773361 1 Входящие 1 2014-03-22T00:02:43 2014-03-22 22846233 3605189702 f3c61a1ee8a8323de4c169db67afbc76 3 504457526759793798 0 3 4 http://mail.yandsearch?text=05&bt http://yandex.ru/pozhet webapteka.ru amalgama 0 0 [125,2919,3852,3467,3755,3849] [] [3] [267] 1846 849 29 8 0 0. 0 0 44 73d7 1 1 0 0 3734681 1 0 0 0 1460 894 157 2014-03-22T06:59:29 0 0 0 0 utf-8 330 0 0 0 8010019903753992422 436843801 0 0 0 0 0 E 2014-03-22T12:38:44 22 1 2 53 16 [4080,637,3085,2188,76,3,260] 2860016566 f4737a6a6d479c3e8f6c174e9422eab4 37171 -1 6 nD a5b3 0 -1 0 2 134 27 0 0 278 302 -1 -1 -1 630 1666 -1 0 0 07d21f 0 [] 0 7540904469640655877 3212143930063194622 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -6483163902225769757 0 журнал «Звезды и россия в машину себя 1 2014-03-23T09:21:53 2014-03-23 1785 611580825 62eb3a585c72f10e638f5217d090aba847 227 2784193748591992768 1 172 42 http://9111147833425073412912706553-pd-1.10 http://vk.com/bu-urun/touchshie_zhizn/zhiznesa/dlya_detes/?r=mail.net/ru/view/2330295776aefe844bHdCMTN3TVF2eGZma1_Kc_Qmun0fTvuCMGW_4PSAIvfP3WjneKtbBGjsbrsk coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 317 346 57 0 0 0 0 5 73d7 1 1 1 1 S820_ROW 1413987 3 15 автобусы для девочек рефератов красивые авито креплено 0 0 592 1623 117 2014-03-23T15:23:10 0 0 0 0 utf-8 401 0 0 0 5087914340569110724 906904915 0 0 0 0 0 E 2014-03-23T19:33:38 55 2 2 8643 14 [10,3,103,42,4,5,47,1,18,9,3993,2,25,14,44] 625776447 7e19356a37bff380c4da31eaa4f601d0 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15981034806496213336 10512538539660077168 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -6502069373992989721 0 Современа - Футбольших 1 2014-03-23T16:23:02 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1903-wh-793-1862278383&c3=ffffffffdf882f80144e5f158fdc http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vcG5ld3MuZ25lemRvcFZ3cXpwRzlKa2tXbWgtWA&b64e games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:37 0 0 0 0 utf-8 401 0 0 0 0 422809779 0 0 0 0 0 E 2014-03-24T07:51:49 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 3111198995473007829 15232020126171326576 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 110 0 -6569123909715121044 0 lady Gang Death project - Раздел - Спортаже и обманулись прямо сейчас бытовой армию: два на легальный 1 2014-03-23T12:37:38 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://9111147811.html?v=1393_0_2513-season_dekor-kupe/mather&page2 http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/sitelink_id=0&year][max]=&search?cl4url=http://lika-disc=0-0-ID8x90.html%26client=gulnogo.net/ru/view=list/?category/gazeta.ru/clck/jsredir?from=yandsearch%2F&ei=J505064327.am15.net/tags/мультики&client=bucking/quick,inline&limit=249&s=1&filter_float_num=600009.html?period=-30&m2=0&output_format=1&cnt=29.0.1878821db3ceb99b664efa093d256e38a1099118%2FbGlua3MmdGltZnBMVE5aSlRfamJlcERHZkFRtUG1VYXNwdEtoUm9IR2ROT3VvSF0WWKAM9MzhkNWJzDQQJTvVdhP0kzZzkzbWt2An5h6DAzjnuV3TTVmaVNmd05QSjF0YVd6MmF4NlpWZE1RcVBiUEtKdlRwUnctTlcydGQyYXo3Uzd05xX1ZgKHRonANzKBQCAB8Oc0iyQBwM&num=224&bid=&filter_float_pre,sitesek/hird-g1467/comment=139504/offererbank-folder:,atta=UlNrNmk5WktYejR0eWJFYk1LdmtxZ2lUR2d3lfbW5iaEx2ZGZhYW1zL2JpZ21pci5uZXQvbmVkJmfX8xIAEsQVgWCVtXe0B_d2ACemhNAhsaeVRof3lzXUlfR0pxCDBeR3t8EQMDQidTAVMiAwQWT3hCVklnY2pWQXImPWF-J28HfURDUyVmZvUHRqVVZoMW5LT1OIej7CRPQ coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 зоофилипуты край ли ферб семьи рисунок найтли криста каранд песню амуравности акадеты на 20.03.2014г. уфе 0 0 1389 884 157 2014-03-23T22:38:44 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 26134642 0 0 0 0 0 E 2014-03-23T22:50:56 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6408603430287046814 4860176597772369010 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 70 0 -6617135007550525549 0 О вветов) для ALCATE&sortf=DT_LAST_PUBLICATE&sort). Регистрации, купить Volvo Sexy 1 2014-03-23T16:14:07 2014-03-23 8221043 1279830590 92b26a315c628ed65b05f41c7c483306e1 225 2861635209668344684 0 82 105 http://yandex.ru/neo2/#messages.yandex.ru/users.html#cmnt http://yandex.ru/cat.php?name":"Соломенять соникс-3м анал на украдуга.рф/idea.com/galler/turne.com.tr/write-avtozvuchatsky-guitar-3SXGLQ854SXQ?sira=7 ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 968 1640 57 0 0 0 0 26 73d7 1 1 1 1 LG Optimus 2179614 3 2 how to copystalk 230 км 0 0 592 1280 209 2014-03-23T06:40:55 0 0 0 0 utf-8 401 0 0 0 0 558315821 0 0 0 0 0 E 2014-03-23T23:37:35 16 1 3 4 0 [72,14,925,9] 1095027162 3d78d8dc2096c7ece3b38cca6d7a6b83 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 7834825270834824731 12435296265939118192 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6685707648291291480 0 Бесплатные жилья и в руковичах 1 2014-03-23T21:20:28 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carfiles/2014/9119&redirektor_view/27409,221&msid http://yandex.ru/clck/jsredir?key=RUR&body=9&page=0&ad_info=ww-1038&blog/list/4.html#/battle/ffffa7eb48b17cc6d28e63\\\\%2Fvideo19838/pWTQtamxDUVlLajl6b0gymebel-pogoda.kz/p/bo4568&form=RurPaymenta.ru/felication_oblast/nice_usd[2]=&transmissional][to]=&drive_serviceId=8489_1366&tsort_offers&group=&is_app_users/skor.com/haberler.ru/volgogradskaya/state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJpNjfFDg3rinFADOKTQ_mAo56EGz3QA-GQV7Yv3dqVNrZRmuDjKoihTN1qGoWb9yiKeN3iZGLV8lxIdiTVh0R0hleFNaZWRXWmQyQzJFdnBGajQzUEdNSG96aGlkYWVtNks0UzZldmNpN21VEbe5WJTDK0m0tA729 docviewer.yandex news.yandex.ru 0 0 [] [443,333,5760] [18] [] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 1 0 0 0 430 384 117 2014-03-23T05:07:54 0 0 0 0 utf-8 330 0 0 0 5177399025069094312 567587409 0 0 0 0 0 E 2014-03-23T13:28:24 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13822395940870043992 4149647501617670018 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 87 0 -6699645891496674711 1 В пробегом - катеристочник 1 2014-03-23T16:21:18 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:51:15 0 0 0 0 utf-8 401 0 0 0 0 563587185 0 0 0 0 0 E 2014-03-24T07:56:38 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 16 22 83 63 0 0 1132 0 -1 -1 -1 1706 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 178 0 -6711080737389998250 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:14:44 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:45:32 0 0 0 0 utf-8 330 0 0 0 0 606583569 0 0 0 0 0 5 2014-03-23T11:15:37 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 -1 0 0 20 84 26 0 471 9 -1 -1 -1 908 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 813 0 -6723139378573305403 0 Современа - Футбольших 1 2014-03-23T16:22:12 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite.html#/battle/ffffff-healty.ru/producers http://mynet.com/porno/page=1&search;web;;%2Fweb%2Fitem%2Ftitle,pos,p0,source,web games.mail.yandex naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:44:30 0 0 0 0 utf-8 401 0 0 0 0 230589762 0 0 0 0 0 E 2014-03-24T07:50:54 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 16147094646787160408 3267175271180936349 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 92 0 -6827875552437106781 1 Дневники | Классной потенцессону озерог. | Европагателе империи 1 2014-03-23T07:31:48 2014-03-23 1785 1197807518 855e0f20066e5f8b529935eca29dda63 4 9117921353016804935 1 56 4 http://911114783/7759&img_url=http://hurpass.com/gazetes http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2410771 3 2 0 0 1366 482 157 2014-03-23T17:00:10 5 1 19832 0 utf-8 401 0 0 0 8959031936800923576 283438122 0 0 0 0 0 E 2014-03-23T19:43:32 0 0 0 0 0 [] 1547029586 c60bbae8a114c907c02e79b3a5e7adbd -1 -1 -1 nD a5b3 0 -1 11 9 12 27 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 10632465148382210160 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -6835310931891846974 1 CAT-форум Первое поедет путевки женщин спально и развлекатеринбурs 1 2014-03-23T15:49:32 2014-03-23 1785 1771120080 7d75e5ce54e6243e703c6d896feff233 196 316119400541676494 1 56 4 http://911114786987.html?html%3Fid%2Fm.hurriyer http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 938654 3 2 0 0 1997 519 157 2014-03-23T01:46:50 0 0 0 0 utf-8 401 0 0 0 8615910193726028779 622662043 0 0 0 0 0 E 2014-03-23T05:01:22 22 2 2 4 0 [] 1272029586 a838ad35997d7a263f34a03154baa8d7 -1 -1 -1 nD a5b3 0 -1 4 3 23 100 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 5960367464715143360 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -6842827632255179584 0 1 2014-03-23T21:11:17 2014-03-23 16862487 1755644572 945af199eed21f3f249577b14f55534b 8363 309245865193991600 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carry_2013.html&forInitQuery=PER_OFFERS_SRCH_MAIN][]=&a1=&a2=30004989614 docviewer.yandex 0 0 [] [443,333,5760] [18] [] 0 0 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 2903448 0 0 0 1 0 0 117 2014-03-23T04:52:29 0 0 0 0 utf-8 330 0 0 1 9110769067793080479 287375675 0 0 0 1 0 E 2014-03-23T13:17:40 16 2 2 0 0 [] 1225767431 2672ee20d87b5c666121145dd76b3644cc 50936 -1 2 2W a5b3 0 1706 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7914288637035474585 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 62 1 -6867873495169505672 1 акте.ру - Лингво-лабора в Новостное авто виды спортал 1 2014-03-23T11:09 2014-03-23 1785 2750008691 d27e873642fdcc142fd52f628470ba33 241 7406071195777269826 1 156 122 http://91111478bcf-ff0001237248a3c684a84763.html http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/view_intext=картинки&clid=1&search.xml?hid=913017&l=map,trfe&trana.ru/23900/?&p=168603021-anons.html?ia=M&lf=F&af=35&s_c=3159_3296_16510865;0;1&x-email.yandsearch/?page=38.03.2014%2F1gEqdtN3LcNMB6CIKsO9mnONRBOu8roZ08buBPgjjNnbUGoe9Y5D3l9VbWj2D9Bt7erMicDOo%3D0%26fh_stream.ru/auto.yandex.ru;yandsearch;web;;%2Fwww.ivi.ru/filmId=rt.ru/saray.html?type_gruzoviki_i_kottelmesi/8926593&group_id=&stribuna-serial/1123413&text=доналдс расписьмо растен пропетро палом лечебный лады в газа коротоколаев метричек фото&nl=4&sp=1&target=2#12 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 13 8w 1 1 0 0 2723983 3 3 puma oxygen sex big tits porno 0 0 1997 547 157 2014-03-23T21:25:37 0 0 0 0 utf-8 401 0 0 0 8546017116003926276 709059014 0 0 0 0 0 E 2014-03-23T21:44:20 22 2 2 0 0 [] 2771758996 1c0ec512cf36858257e85c74d2ac7654a2 -1 -1 -1 nD a5b3 0 -1 0 61 100 108 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 9421855928520292696 11319102890696173680 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -6880179944191362326 1 В пробегом - катеристочник 1 2014-03-23T16:12:53 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T05:48:42 0 0 0 0 utf-8 401 0 0 0 0 847205448 0 0 0 0 0 E 2014-03-24T07:51:46 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 95 93 793 0 5720 0 -1 -1 -1 5017 3724 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -6986728922972248352 0 Женская библи в шопогрузки 1 2014-03-23T18:17:14 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/forums.kuba.com/kampaign/files http://yandex.net/ru/video/search yandex.ru tury.ru.msn 0 0 [85,350,1212,3849,8358] [106,323,1092,3676,13188] [18] [28] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 -1 0 0 0 592 503 117 2014-03-23T19:18:45 0 0 0 0 utf-8 401 0 0 0 0 1070533242 0 0 0 0 0 E 2014-03-23T12:25:44 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 1477 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6372803000235980495 925019927264479664 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -6998847403379936884 0 1 2014-03-23T21:14:59 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:45:42 0 0 0 0 utf-8 330 0 0 1 0 606583569 0 0 0 1 0 E 2014-03-23T11:15:48 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 460 -1 -1 -1 -1 -1 -1 -1 -1 6631 5214 942 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 584 1 -7253547937523001925 1 Чтений найдено в Яндекс.Музыка" - Свежие сотказать бу авторов 1 2014-03-23T05:29:05 2014-03-23 1785 1739349077 d1e5e976bfd136ed9cad4fdb3524268e 102 1930034221481539513 1 156 4 http://911114784E20437b406b71056-mon-e-n-s-mesjachok http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 1088451 3 2 0 0 2011 877 945 2014-03-23T22:25:39 0 0 0 0 utf-8 401 0 0 0 5594739182691462682 576508429 0 0 0 0 0 E 2014-03-23T17:26:38 22 2 3 0 0 [2,1019,3993,9,14] 1650083551 0b32fa88d354c65c66315c662d206ac906b2 -1 -1 -1 nD a5b3 0 -1 9 172 243 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 13193074755622332528 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7395638780722795911 0 Современа - Футбольших 1 2014-03-23T16:23:10 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1830520[]=2&FILTER[32685626%2Fb%2Ffotogram/18 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20udWElM0QlM0RmZVNmZ1NNVzJyZkNmxFafzTGrazFsbKeQXfHEFzFfLycbGiwgeyVqAFM8Hj0hFn56d1xSX0ZOcwkxX0V5fAZR games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:49 0 0 0 0 utf-8 401 0 0 0 0 1015390617 0 0 0 0 0 E 2014-03-24T07:52:02 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15232020126713526616 11875920749657778288 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 122 0 -7407752634189615759 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:57:42 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,27,Zhitovit.com/iframe http://yandex%2F15551&lr=76&text=обезьянка haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:44 0 0 0 0 utf-8 401 0 0 0 0 663421629 0 0 0 0 0 c 2014-03-24T07:38:25 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 17131 -1 2 wi a5b3 0 -1 0 0 52 15 0 0 272 0 -1 -1 -1 401 -1 -1 0 0 07d21f 0 [] 0 713690030010978492 14503036338486044784 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -7479382069659086597 0 Продаю бесплатно - светике Татья владельцев 1 2014-03-23T21:20:45 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://kliksa.com/iframewom http://galerid=0&high-anner.by/?search?fr=all&text=купить доме&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgo moniac.com.travellecomme exist.ru.msn.com 0 0 [292,353,7606] [5760] [18] [28] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 5 0 0 0 430 384 117 2014-03-23T05:08:11 0 0 0 0 utf-8 330 1 0 0 5177399025069094312 567587409 0 0 0 1 0 E 2014-03-23T13:28:39 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2843856621567445410 2250008690469188482 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 98 1 -7516343859483962604 1 Автомобили с проблетов онлайн на 1 2014-03-23T05:48:22 2014-03-23 22447063 3721844867 7e52689e524f80aac08d88e47e84e73d 9580 2301018798038122137 0 56 4 http://images.yandshijj-v-cheap-and http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1297 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3238011 3 2 0 0 653 693 117 2014-03-23T07:41:35 0 0 0 0 utf-8 330 0 0 0 4627722567800183582 485309851 0 0 0 0 0 E 2014-03-23T04:08:48 16 2 2 0 0 [] 3446251501 53d889ddb55859ed8adbe55c5cda7da177 45687 -1 1 wi a5b3 0 -1 17 14 261 21 0 0 1117 171 -1 -1 -1 2495 -1 -1 0 0 07d21f 0 [] 0 4313994922287858055 1100934878990743947 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7580359850907242723 0 продажа консультате с руков на AVITO.ru - страни в портименко 1 2014-03-23T19:23:45 2014-03-23 12591643 3565819545 82af9c35e16ca87ffaf7b9edfa6f42f6 239 2189462870885553765 1 97 42 https://mptri.com.tr/tr/videobox.tv/eventsnazorva/sevastok_2838##name=Новоставить http://yonja.com/etgifrm%2Ftitle,pos,p0,source,web,pre,site_slotname=Tümü&e_baski-gazetable_key=&price_valka_277565/?from=0m7g&clid=1965}/catalog%2Fsocial_hastnye-prizatsii/adme.ru/?clid=1&metrikansii/bez_opyta_sport-expresoriends.ru/ru/clck/jsredir?from=29.03.251629804b21hR1gwMGZyU013JTNEZnZTTVRBJTNEZmRTTVUUT2gtZEJhd1hHJKEMXiKRz8iFPth adultmultiki yandex.ru 0 0 [353,3121,11503,11630] [9,377,480,3676,15216,14241] [18] [28] 968 1640 57 0 0 0 0 7 73d7 1 1 1 1 LG Optimus 2853155 3 2 don шнуровногорий шёлк щёлк 0 0 592 547 296 2014-03-24T07:46:25 0 0 0 0 utf-8 401 0 0 0 8608824822004667778 816585860 0 0 0 0 0 E 2014-03-23T11:06:01 22 2 2 106 31 [397,11,9,68,14,1019,1465,1,3993,252,2] 3702300762 a18b5c7211334fac99217724417d4550d8 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 17339020080848845144 8629326841291084175 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7599913264897309014 1 Отступным низкой области в Киеве. Решение пенсиональный фото доставни.ру — Яндекс.Музыке. Мне на AVITO.ru - Продаж - визажиганы - Мы от Good, B-russianFood.com: новости на новости на 20 формация 1000005 года, долла бывший чай - НОВОСТИ - 20 купить бесплатный с персонские первый заявки деловек птичка 1 2014-03-23T12:22:58 2014-03-23 1785 1353429726 e2c0e129718aad4e93f93b5c5c066ed675 3 6510982234989222954 1 105 7 http://9111147834-video-domasma-hd.net/best-1182761198 http://yandsearch;web;;%2Fwwwww.silverdi-kading/sex/page=310&text=фильмы онлайн штор&client_id=599277a696313/album/17-yil-hake.ru/?/ coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 3701 1667 29 8 0 0. 0 0 5 qR 1 1 0 0 2765439 3 2 0 0 2793 1041 157 2014-03-23T20:25:05 0 0 0 0 utf-8 401 0 0 0 9043742639069289622 56687838 0 0 0 0 0 E 2014-03-23T22:58:27 16 1 2 0 0 [] 1764751978 72253090507ab4406a6d83717c75243e -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13306854087717566808 2241902997682127110 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7687055982568657638 1 Историентов на AVITO.ru - Поттехнику в асоціальный сайт На юго-востинга" : один бытового парфюмерили новости, дереводы. Realty.dmir.ru - Почта Mail 1 2014-03-23T16:58:38 2014-03-23 33044186 2126080452 fab776b7bdcf8dcb68cb6922724a9362 89 2034549784946942048 1 3 5 http://yandex.ru/cgi/online.com.ua/detskie-spb.blizko.ru/index.ru https://market.yandsearch/non.ru top-androeed rabota.yandex 0 0 [3,15,63,95,75,381,2182,4132,10886,16137,15261,13770] [2,15,46,123,102,507,3498,4504,15216,10157,15095,11161] [18] [28] 1846 1640 57 8 0 0.77 3 6 10 73d7 1 1 0 0 2175425 -1 0 0 0 1071 955 945 2014-03-24T11:26:33 5 1 19832 0 utf-8 401 0 0 0 5669772058100890221 888201737 0 0 0 0 0 E 2014-03-23T06:10:57 39 1 3 8971 0 [95,6,408,103,10,3,44,42,76,47,3993,209,51] 1341355226 fdd940e1982462885d4c8fb848816939 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15349291019717589336 8037358779388095153 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 0 0 -7780035710352963112 1 Голая Нагородской областической 1 2014-03-23T05:23:08 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/load.ru/#hs http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T05:29:56 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 546201586 0 0 0 0 0 5 2014-03-23T03:47:32 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 3 wi a5b3 0 -1 0 0 277 84 0 0 858 85 -1 -1 -1 1454 994 -1 0 0 07d21f 0 [] 0 4313994922287858055 18236259282679687280 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 10 0 -7790500540215037749 1 Тимошенковске на F1news: Новгородажа легководителей и ее европы - Странспорт 1 2014-03-23T10:51:56 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.139517&quickpantinoyleri http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 943 296 2014-03-23T22:22:46 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 515139757 0 0 0 0 0 E 2014-03-23T20:22:37 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 25 116 83 126 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 7982945165681148628 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -7794706186709683946 1 В пробегом - катеристочник 1 2014-03-23T16:32:20 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:59:55 0 0 0 0 utf-8 401 0 0 0 0 667659827 0 0 0 0 0 E 2014-03-24T08:03:47 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 49 20 137 36 0 0 1861 0 -1 -1 -1 1987 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 236 0 -7829278250573826232 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:56:49 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:02 0 0 0 0 utf-8 401 0 0 0 0 708852166 0 0 0 0 0 E 2014-03-24T07:37:21 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 22279 -1 1 wi a5b3 0 -1 0 0 1 28 0 0 559 0 -1 -1 -1 1476 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -7893175662663208254 0 1 2014-03-23T21:20:38 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carfiles/2014/9119&redirektor_view/27409,221&msid docviewer.yandex 0 0 [] [443,333,5760] [18] [] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 0 0 0 1 430 384 117 2014-03-23T05:08:04 0 0 0 0 utf-8 330 0 0 1 5177399025069094312 567587409 0 0 0 1 0 E 2014-03-23T13:28:33 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 314 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4149647501617670018 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 116 1 -7939467002160261687 0 1 2014-03-23T21:12:27 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogo_baskalabs.html tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:43:02 0 0 0 0 utf-8 330 0 0 1 0 529683494 0 0 0 1 0 E 2014-03-23T11:12:54 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 1 nD Tp 0 255 -1 -1 -1 -1 -1 -1 -1 -1 2253 3940 48 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 2665691200259067948 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 993 1 -7940922169703095731 0 Знакомства в Республике Дагестанцам 1 2014-03-23T11:36:25 2014-03-23 1785 994141745 9da519855c6298ca2b15a5da579246a283 207 1836964949227567248 1 42 71 http://9111147851/?&p=1&text=укладоватки закон http://yonja.com/kamuajanssories.ru/ulya-volt.ru/kino-haber.com/peugeot/230000&sp=-1&o=likest.tanka.pro/calendi-shpartaming%2Fsimferotiv-vosti.ua/oldcars coccoc.com yandex.ru 0 0 [3,5,6,15,57,91,76,175,353,403,349,3121,11503,11630] [119,3418,2455,3036] [18] [155] 1009 1367 57 0 0 0 0 0 73d7 1 1 1 92 iPad 7 1443169 3 2 психологда тура кирпич 0 0 674 1115 117 2014-03-23T19:40:39 0 0 0 0 utf-8 401 0 0 0 0 732369666 0 0 0 0 0 E 2014-03-23T02:32:34 55 2 2 0 0 [9,11,42,3,1] 990921201 f7aaf7453dcba45c62882493cd80e6fe81 -1 -1 -1 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 14359677112441125208 16829396780134885488 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7960500408286544976 0 Дикие стал автомобильники | каталог холстук, баз для мультуры по которы, услуг Россию :: Ходческие, фильм "Холодная рабочий купить 1 2014-03-23T21:20:31 2014-03-23 22422732 2271852515 56b72cde5096c0f237452cd9f670d92c 196 8579128732000753997 1 223 42 http://video/torre.com/odnoi-stimeout=1440&is_vative=0&s http://fast-topic/282039.html%3Fid%2F1000/query.ykt.ru/yandex.ru/fblogs.yandex.ru;yandsearch ficbook.net mail.yandex.ru 0 0 [5,4,372,9487] [15] [] [28] 968 1640 57 0 0 0 0 4 73d7 1 1 1 1 LG Optimus 1769215 1 0 0 0 592 838 157 2014-03-23T06:41:05 0 0 0 0 utf-8 401 0 0 0 4898301856994370214 460298661 0 0 0 0 0 E 2014-03-23T13:28:03 16 1 3 3918 6 [64,7,6,109,14,5,285,84,81,80,4] 2295410265 b140e0077981e3689a5f6973035a3b7e -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10671434186064935256 1759711880979997785 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7972070184216637013 0 Honda Bellas - Jerry - моя стулья | Новини 1 2014-03-23T21:12:45 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/bagajd-world/567765647 http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:43:26 0 0 0 0 utf-8 330 0 0 0 0 274009037 0 0 0 0 0 5 2014-03-23T11:13:16 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 132 174 11 0 462 6 -1 -1 -1 543 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 2948755971896419986 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 656 0 -8035654727346356734 1 В пробегом - катеристочник 1 2014-03-23T16:27:45 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T06:02:54 0 0 0 0 utf-8 401 0 0 0 0 426221199 0 0 0 0 0 E 2014-03-24T08:04:20 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 46 119 0 0 740 0 -1 -1 -1 1412 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -8075310065356086365 0 Hotel.jpg» на продаю карте | | Для домов.НЕТ - поиск, познавала «Весен 1 2014-03-23T21:15:10 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/ball/anasayfa.irr.ru/page=0&availaut/to/casino http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:45:57 0 0 0 0 utf-8 330 0 0 0 0 11121749 0 0 0 0 0 5 2014-03-23T11:15:56 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 927 314 11 0 282 13 -1 -1 -1 1159 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 15729620011142801520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 658 0 -8094433118608774279 0 Светские главные, долла: упражные сериалы доллар - Страница 10 сон. Женскую система Rezervative Burcundai Sanatik 1 2014-03-23T16:33:25 2014-03-23 18645263 1790084893 60057a4e582b5faf88f2c49235bc138a 8363 803723286935347333 0 42 71 http://blog/resim-6/ http://yonja.com/quests forum.shestvennik.ru yandex.ru 0 0 [353] [] [] [28] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3031088 1 0 0 0 592 1376 117 2014-03-23T07:39:47 0 0 0 0 utf-8 401 0 0 0 5718608483174516921 922710799 0 0 0 0 0 E 2014-03-24T01:25:47 16 2 2 0 0 [] 1983574728 e3ebee64fe91325ffa29ba8579b9d3ae 62835 -1 1 wi I7 0 -1 0 0 0 7617 0 0 5486 984 -1 -1 -1 15108 -1 -1 0 0 07d21f 0 [] 0 2351151346570785378 7780118309660496456 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8149355527847772674 1 В пробегом - катеристочник 1 2014-03-23T16:24:43 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:54:01 0 0 0 0 utf-8 401 0 0 0 0 778610602 0 0 0 0 0 E 2014-03-24T07:58:46 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 16 22 61 62 0 0 785 0 -1 -1 -1 1064 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 173 0 -8155806667442330296 0 Hotellot - Рыбалка.ру - все серия 1. Писточный или Турция, Гомельной банка 1 2014-03-23T21:14:08 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/ball/image&uinfo=ww-135136361 http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:44:44 0 0 0 0 utf-8 330 0 0 0 0 44551634 0 0 0 0 0 5 2014-03-23T11:14:53 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 184 499 18 0 779 7 -1 -1 -1 1948 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 5463880999052126868 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 823 0 -8156909267415167339 1 Брюки с TurboBit.net - фильмы онлайн на 2014 - Финансий 1 2014-03-23T16:23:08 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://hurpass.com/ogrevolyni http://foto-395077193b7240786&lr=11110436606 photostrana cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [9,121] [] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:47:01 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 845095662 0 0 0 1 0 E 2014-03-23T23:47:08 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 51 2009 1293 2 702 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 18315239292404426864 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 1 -8157755090657987777 1 вк — Яндекс.Почта/АndroidMag 1 2014-03-23T09:50:56 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.1401/sovmestore http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 637 296 2014-03-23T21:17:04 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 834077893 0 0 0 0 0 E 2014-03-23T19:47:34 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 11 77 91 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 9450665378893719664 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -8165890628220817396 0 1 2014-03-23T21:13 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/bagaji-demoi-fena tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:43:36 0 0 0 0 utf-8 330 0 0 1 0 274009037 0 0 0 1 0 E 2014-03-23T11:13:27 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 5 nD Tp 0 204 -1 -1 -1 -1 -1 -1 -1 -1 2055 3158 320 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 855065554003365461 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 766 1 -8207621839602888071 0 21:24:02) « Политик 1 2014-03-23T18:06:44 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/for/zilcc.ru/tsotsbank yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 0 0 0 0 430 234 117 2014-03-23T19:08:37 0 0 0 0 utf-8 401 0 0 0 0 131924827 0 0 0 0 0 E 2014-03-23T12:13:53 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 3912 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 6785386778629335136 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -8262700426099359718 1 1 2014-03-23T16:21:13 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://video/embed/68411756114414293 http://foto-395077193b7240786&lr=11110436606 amksports.ru.livemaster cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [45,333] [18,348] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:44:38 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 42215400 0 0 0 1 0 E 2014-03-23T23:45 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 49 5647 4852 2 2891 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 2532179236054953957 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8321182583543853388 0 1 2014-03-23T20:34:29 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 tranamaschools 0 0 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 0 0 0 1 1 1 117 2014-03-23T06:05:11 0 0 0 0 utf-8 401 0 0 1 0 122853544 0 0 0 1 0 E 2014-03-24T14:17:35 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 2998 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8456596711470618355 0 Samsung Galaxy tarisme online Search resmi Sistem TurboBit.net.ua - DX 130 000 рубля игры 1 2014-03-23T06:47:03 2014-03-23 1785 901953317 a4a276d0ea345c74148750a6919a36dad5 207 1836964949227567248 1 42 71 http://rutube.com.tr%2Fgazpromo38.ru/search?lr=2&rpt=simage avito.russia.rust.net.ru 0 0 [] [3,160,864,2455] [] [] 1009 795 57 0 0 0 0 0 73d7 1 1 1 92 iPad 7 1443169 0 0 0 0 674 927 117 2014-03-23T14:25:27 0 0 0 0 utf-8 401 0 0 0 0 833653080 0 0 0 0 0 E 2014-03-23T19:03:13 55 2 2 0 0 [9,11,42,3,1] 1061339287 20bfc7e7245325fdac7e99f96f7dcbe6 -1 -1 -1 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 13975993063278569584 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -8472838923458193599 0 Личный компаний: "В бороны панели 1 2014-03-23T10:26:41 2014-03-23 1946 2822174503 83f4c4017c625c30615e808b9b553cd25a 950 1989946518134869356 1 42 71 http://avito.ru/shoes-with-avciliusersView yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 1 0 0 0 0 592 1376 352 2014-03-23T04:42:49 0 0 0 0 utf-8 401 0 0 0 0 285947293 0 0 0 0 0 E 2014-03-23T02:39:02 16 2 2 66 0 [2193,82,265,14,95,6,100,9,72,3275,1,70,11] 3219490004 5c6e7add6158bbed0699cbe973042ea2ef -1 -1 -1 nD Tp 0 -1 0 0 0 2817 2921 2527 30475 0 -1 -1 -1 29211 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 15977284023899318384 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -8555937822471823535 0 1 2014-03-23T16:34:48 2014-03-23 18645263 1790084893 60057a4e582b5faf88f2c49235bc138a 8363 803723286935347333 0 42 71 http://blog/resim-6/ forum.shestvennik.ru 0 0 [] [] [] [] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3031088 0 0 0 1 592 1376 117 2014-03-23T07:41:33 0 0 0 0 utf-8 401 0 0 1 5718608483174516921 922710799 0 0 0 1 0 E 2014-03-24T01:27:26 16 2 2 0 0 [] 1983574728 e3ebee64fe91325ffa29ba8579b9d3ae 62835 -1 1 wi I7 0 2002 -1 -1 -1 -1 -1 -1 -1 -1 25844 25608 85 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7780118309660496456 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8563276039202285702 1 Голос видео 1 2014-03-23T05:23:15 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/triket/4746734017811fc051dd0e57 http://yandex.ru;yandex.ru/?area=ShowForum.ru/look/event cdo.e1.ru.plays олины.рф 0 0 [] [] [] [] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 -1 0 0 0 1460 549 574 2014-03-24T05:30:03 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 215720129 0 0 0 0 0 5 2014-03-23T03:47:38 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 5 wi a5b3 0 -1 0 0 289 1 0 0 94 20 -1 -1 -1 488 414 -1 0 0 07d21f 0 [] 0 18236259283221887320 16190469452247787632 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 9 0 -8599500047480121116 1 Брюки с TurboBit.net - фильмы онлайн на 2014 - Финансий 1 2014-03-23T16:22:36 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://hurpass.com/ogrevolyni http://foto-395077193b7240786&lr=11110436606 photostrana cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [9,121] [] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:46:24 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 42215400 0 0 0 1 0 E 2014-03-23T23:46:39 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 18315239292404426864 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 1 -8602651835226449867 1 7 самая мамки для ваше может по суперма Страница 719 тыс. ответов 1 2014-03-23T12:16:45 2014-03-23 1785 2088542490 921202579dbab4e58eddb04f693854b3 57 9499572962667875395 1 56 122 http://911114788/38/celebekgaleri.hurriyet http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/downloadTo=&orderby=RelAmountry_id=22&ved=0CAUQjB0&url=http://money?rand_cruitstart=444305.952058.13951639370f280c133ad16ce2c79e7cab93c5a23X3 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 11 6ee9 1 1 0 0 2366248 3 3 схема сумерки 5-6 летовой устанния 0 0 1627 708 157 2014-03-23T22:11:02 5 1 24018 0 utf-8 401 0 0 0 8929720244896745512 460839648 0 0 0 0 0 E 2014-03-23T22:21:38 0 0 0 0 0 [] 114378192 d7364e6a7ddcbcf6dcccca7bd6b2807a -1 -1 -1 nD a5b3 0 -1 0 0 626 189 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6784513853171268256 7631625543297538199 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8607066140185696713 1 Отслеживаны - №1065 - ekşi 1 2014-03-23T16:28:07 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/weatherson.html#78efe9869.shtml?wmid=143&srcid=UxB0pAAA6I9d0CWaWE%3DfrSMw http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T06:03:11 0 0 0 0 utf-8 401 0 0 0 0 813102074 0 0 0 0 0 E 2014-03-24T08:04:40 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 15 110 78 102 0 0 2060 365 -1 -1 -1 2397 1361 -1 0 0 07d21f 0 [1555356] 0 6331778101536488133 11618756116771170416 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 8 0 -8611399304547034181 0 Современа - Футбольших 1 2014-03-23T16:22:51 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1188c56ff8058343682.1_29_et._30108879a9aa61ea73752719 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vdmlyLnJ1Lw%3D%26engine_volume2=40&sid=6&ved games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:23 0 0 0 0 utf-8 401 0 0 0 0 356217277 0 0 0 0 0 E 2014-03-24T07:51:41 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11767694963832011096 2937255180427836822 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 111 0 -8664317093542350977 1 Афиша - купить готовой менение сиUçuşlardo, Yazlık Burçlar - astopol.info - Classion AAAQ (250Hp) CVT (140 тыс. км., смотр объявлению в автобусов - Екатегория России, клиент 1 2014-03-23T20:34:25 2014-03-23 32821467 1157253311 482b48fd303d64bba86ec7965c6ed6dd67 207 2460387875241374418 0 56 4 http://rabota.ru/show%2F2014 http://read.php?act=8&ved=0CC85ViwdY8L4WG9pLWlKS0RJU3AeAxJuh wwww2.sports m.auto.yandex 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3815009 -1 0 0 0 1136 593 117 2014-03-23T06:05:10 6 0 39566 0 utf-8 330 0 0 0 7635035874316412186 1000586617 0 0 0 0 0 5 2014-03-24T14:17:28 39 1 2 70 51 [10,42,1,22,3,112,3773,103,5,51,15,4,3212,47] 1549383847 ed3b609e3397b12c56cca4469c881c38 17234 -1 7 wr a5b3 0 -1 0 0 179 57 0 0 212 103 -1 -1 -1 613 -1 -1 0 0 07d21f 0 [] 0 3832538266798636143 15412096237897967728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 22 0 -8665395302091187491 0 Современа - Футбольших 1 2014-03-23T16:22:43 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=106/cid=191&rpt=simages%2Fthumb%2Fimage&lr=157 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20v&com_id=0&body_type=0&clid=1980&s_tg=&tariniz games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:12 0 0 0 0 utf-8 401 0 0 0 0 588577730 0 0 0 0 0 E 2014-03-24T07:51:28 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 1560098688466543352 11767694963289811056 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 121 0 -8673158852214232182 1 Голая Нагородской областической 1 2014-03-23T03:51:08 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/load.ru/#hs http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T03:26:28 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 270908049 0 0 0 0 0 E 2014-03-23T02:24:56 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 55446 -1 3 wi a5b3 0 -1 31 93 484 10 0 0 546 176 -1 -1 -1 1135 -1 -1 0 0 07d21f 0 [] 0 4313994922287858055 18236259282679687280 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -8725981425210147796 0 Аудио ЭХО Москве, ножений вопросова нашлось 397 1 2014-03-23T02:41:19 2014-03-23 15665476 1648690379 04eb9547b256147702f34bd2464beaf0 196 1425686864647023563 1 56 3 http://ivona.ru/top.rbc.ru/moscow=&q= Erection27=&toName=false&morpholog/vakansition&ref=city/daily http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/news.rambler.ru/search/ex09KBBcpWRDlaYTZYNEJHZDlOb3VHdjNWRXg5UFBpV2pFQ1qUQ&b64e=2&output=uds_app_id=19855479&view?ad=44&s_yers.don.ru/context=Sex&geo=0&s_tg=&offset=2.\\tПоповорого план кубе&clid=20639&keyno=0&l10n=ru&mc=1.58496250001978934&Lt=6&refresh=1&has_photofunia.ru/#!/video/search;web;;%2Fweb%2Fitem%2Ftitle,pos,p1,source=web&text=текстра фото 2014/view/125329d08a194e758644-500h_bWOg&bvm=bv.62922401,d.bGE&cad=rjt market.ru.platnye-igrydlja tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [] [] [28,267] 1622 1583 29 8 0 0. 0 0 10 73d7 1 1 0 0 1808122 3 3 интернет магазин финес и физическу о приколеснока, валерия 0 0 2011 768 157 2014-03-23T04:34:29 5 1 19832 0 utf-8 330 0 0 0 6243379005186084238 8777098 0 0 0 0 0 E 2014-03-23T21:50:32 16 2 3 854 0 [2193,82,265,3275,347,14,72,925,100,3,80] 1245455042 891bf70623c57bfdd5e9d09d616390eb -1 -1 -1 nD Tp 0 -1 0 72 206 84 12371 0 -1 0 -1 -1 -1 18 -1 -1 0 0 07d21f 0 [] 0 2560641399298933162 12328847313983957104 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8746280778383767298 1 Поздравоохранул 1 2014-03-23T13:55:17 2014-03-23 1785 3609570631 e9fd65b19c652c4ce85594f2fcd0c7db 196 593475812248875581 1 227 105 http://911114783342507357617c9eb279bb4f2642 http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/video/index.ua/pole-2167ba0f7c355305-nam-v-pid2-290622263757&m=3,5&Volume[1]=&extras[24]=0&eae=0&clid=22876.26.2.0.html&docid=onair.com/galeride/Kariy_el/zafiruet-onlajn_tr_5Gn0cFj_bANt_S8ROdvfrOg6pCgU5XY2P3MtHkZQNDILDhspeAAhWjJJgkFq5HXGrEC5GmjXQ5bGV3TTVeklJakNwelhKc0hDcjFTVTFWb180ff0d9&url=http:%2F%2Foktavito1296_3346d0f0;dcid=ya_wireleva.ru/yandex.ru/?action=page-67600&groups[]=17704&groups[]=13&msid=91cc2424241d2cd8128&lr=97541¬custom_key=&haggle_AO_Mondelevizit-dvigator.tv/tnt/?region=cadf2922401,d.bGE&cad=rjt coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 523 617 57 0 0 0 0 44 73d7 1 1 1 0 3684816 3 3 алекс старт 20кг купить десан автобазаречь создать 0 0 592 310 157 2014-03-23T22:38:38 0 0 0 0 utf-8 401 0 0 0 8382449486543033592 279598542 0 0 0 0 0 E 2014-03-23T03:43:12 16 1 3 119 4 [76,1] 3886548777 f69565e66e18aeaf2059a8b6495c723613 -1 -1 -1 nD a5b3 0 -1 0 28 22 9 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 4244945004353045279 5931335146677711795 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8931522039698213139 1 В пробегом - катеристочник 1 2014-03-23T16:28:08 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:56:58 0 0 0 0 utf-8 401 0 0 0 0 126947969 0 0 0 0 0 E 2014-03-24T08:00:54 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 22 23 62 61 0 0 996 0 -1 -1 -1 1043 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 240 0 -8933906015862790098 1 Афиша - купить готовой менение сиUçuşlardo, Yazlık Burçlar - astopol.info - Classion AAAQ (250Hp) CVT (140 Hp) 2006 купить компании Украина вытащил о Москвы :: Голосуток в 1 2014-03-23T20:34:36 2014-03-23 32821467 1157253311 482b48fd303d64bba86ec7965c6ed6dd67 207 2460387875241374418 0 56 4 http://rabota.ru/new.mcko http://read.ru/yandsearch wwww2.sports m.auto.yandex 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3815009 -1 0 0 0 1136 593 117 2014-03-23T06:05:24 6 0 39566 0 utf-8 330 0 0 0 7635035874316412186 28170112 0 0 0 0 0 5 2014-03-24T14:17:36 39 1 2 70 51 [10,42,1,22,3,112,3773,103,5,51,15,4,3212,47] 1549383847 ed3b609e3397b12c56cca4469c881c38 17234 -1 7 wr a5b3 0 -1 0 0 106 58 0 0 92 34 -1 -1 -1 148 -1 -1 0 0 07d21f 0 [] 0 15412096238440167768 13753603333183694960 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 19 0 -8939870126547196026 1 Комная прессан Партира, акциях — фанфики к экзаментально и части Донецка | РБК daily Digital acro 1 2014-03-23T16:02:25 2014-03-23 1785 2922543887 1c01b17e6ee3c1a01828ecd318a6b581 59 8723854704339558313 1 56 2 http://9111147842,0.01178.1394999-prepovedeki http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net%2F1b.jpg","photos&p=3&page=18413930_handguides[]=2&sign=cf8f01c12489e0a7&uuinfo/view/93190876323017&l=sem&sign=213&nore[2]=0&search?p=законы финальных треуголоваров владимирова туалеты 10 дневники андроид&lr=20525944&lr=1637&keyno=0&room=0&page=576&m[]=100&search/retsiatoust/avtobank.ru/#!/kemeye-karechenkoi-denegro/brando.ua%2F17%2F204a%2F&ei=0-ochekiev.ua/opinion%26wheelpUTdBQUhtbTZhSzItYVZOOUjhfNF9vZDdGVzLzAwMzA3L3BwLzQxNjY2ZHpnY2VWxiCHsuJClHPnYMQVoxbSVTUFJTElsakI4WFp6NHNHJRDRhOU1LR3BuSmJYYj9tOUmxUOXMtVlN1TjNtVGc1c032a1b3672 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1297 1010 57 8 0 0. 0 0 18 73d7 1 1 0 0 2992777 3 3 детский номер электрообная игры скорота фото кало 0 0 1366 391 157 2014-03-23T00:56:51 0 0 0 0 utf-8 330 0 0 0 4847654860042290847 969371126 0 0 0 0 0 E 2014-03-23T02:42:26 55 1 3 12257 0 [10,9,112,68,365,76,260,22,1,2817,3,3540] 2380200681 0735dfb043075c30693475487bf24a49b7 -1 -1 -1 nD a5b3 0 -1 1 55 84 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 5849787649459211608 17662636599668303984 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -8964700706069591280 1 Салата 10. Цензор видео, онлайн (все забилей с пройти в Мельного 1 2014-03-23T04:19:17 2014-03-23 22447063 1796001934 3ae86dda5556b7f4ef38aa8077f6825a 15887 1644674872295047497 0 3 4 http://images.yandex.ru/neo2/#inbox/&dates=0&run[1 http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 2 [353] [] [] [28] 1587 555 57 10 13 0.77 0 0 25 73d7 1 1 0 0 3996694 3 2 0 0 1713 555 117 2014-03-23T19:14:50 0 0 0 0 utf-8 401 0 0 0 0 939725270 0 0 0 0 0 E 2014-03-23T02:46:14 0 0 0 0 0 [] 1907295579 dc652869f8d8eff9ed5b5daa5374b163 5038 -1 3 wi a5b3 0 -1 305 0 452 30 0 0 4029 164 -1 -1 -1 5023 6719 -1 0 0 07d21f 0 [] 0 4313994922287858055 9730402382055663728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -9065033574850035452 1 Серия 9 - 2013. Дети@Mail.Ru — социалист 1 2014-03-23T16:20:14 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://yandsearch?lr=35&rpt=simage&lr=66&text=розаврилова http://google.com/games.mail.ru ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 3 2 0 0 1994 561 117 2014-03-23T06:43:38 0 0 0 0 utf-8 401 0 0 0 9007862211105217806 42215400 0 0 0 0 0 E 2014-03-23T23:44:06 16 1 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 47 40 590 134 0 0 2110 -1 -1 -1 -1 -1 3460 -1 0 0 07d21f 0 [] 0 1930741556500475038 9207967286803898539 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -9186659792231785281 1 Настройствоваться частных (б/у) автор о продукции; развития детьми - Грузов 1 2014-03-23T09:48:08 2014-03-23 1785 1341248658 6366e773993d35514d6b846f79b34292 183 626923241082458450 1 56 4 http://9111147832977565.html?cat=420801;label=perioda http://go.mail.yandsearch coccoc.com yandex.ru 0 0 [3,5,6,15,57,91,76,175,353,403,349,3121,11503,11630] [119,3418,2455,3036] [18] [155] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3500889 3 2 0 0 1997 548 117 2014-03-23T17:20:17 0 0 0 0 utf-8 330 0 0 0 8416052423457712746 215289560 0 0 0 0 0 E 2014-03-23T23:14:07 16 2 2 13621 14 [14,72,80,925,370,109,7,285,3274,101,6,66] 1160820115 524ee8575739a6149a641e6f4fbc6f7b -1 -1 -1 nD a5b3 0 -1 9 45 109 42 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8578740285396261239 4548538545130905100 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 - --- !sql13 -- -4632802378222380466 1 null Acoper «labilir mitsubishi в Липецке на Batak 1 2014-03-23T16:23:07 2014-03-23 27528801 2723656327 7fc6e55c74e3029834ae16bf67d8cb9f4f 15887 2224216775939367616 0 56 4 http://rutube.ru/patianu http://love/v012/04 footki.yandex.ru m.fotoalmeta 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3238011 -1 0 0 0 1136 555 117 2014-03-23T06:04:09 5 1 19832 0 utf-8 401 0 0 0 0 1018986580 0 0 0 0 0 5 2014-03-24T08:02:03 16 2 2 0 0 [] 2529497135 30d877a66ec8945d1e4d056e51e3de35 18372 -1 2 wi a5b3 0 -1 4333 1332 747 2472 0 0 7395 285 -1 -1 -1 16115 -1 -1 0 0 07d21f 0 [] 0 307141980878902893 338497349483004835 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 47 0 -4716566754160586468 1 Тимошенковске на F1news: Новгородажа легководителей и ее европы - Странспорт 1 2014-03-23T10:52:43 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.139517&quickpantinoyleri http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 943 296 2014-03-23T22:23:20 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 278406132 0 0 0 0 0 E 2014-03-23T20:23:36 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 0 0 1 2 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 7982945165681148628 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -4755557211339260600 1 PwC 1 2014-03-23T16:25:12 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/index?appkey=&m[]=6&frommail.yandsearch?text=никол в про и невиннисаж http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T05:54:31 0 0 0 0 utf-8 401 0 0 0 0 346099590 0 0 0 0 0 E 2014-03-24T08:01:40 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 59 78 0 0 2251 0 -1 -1 -1 3362 798 -1 0 0 07d21f 0 [1555356] 0 6331778101536488133 6157439411447496521 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -4770046483886760822 1 Mercedes-Benz e-клас. Можно купить 1 2014-03-23T16:26:50 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/weatherstyle/29/127737064&domain=hurriyet.com/volgogram/22/10/Default-tistings=535674/atturkiye http://mynet.com/mate-guzellileridetay/80508/bio/video?q=породажа хабаров с технологических mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T06:02:18 0 0 0 0 utf-8 401 0 0 0 0 782597291 0 0 0 0 0 E 2014-03-24T08:03:20 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 96 115 252 0 0 6165 306 -1 -1 -1 8173 1287 -1 0 0 07d21f 0 [1555356] 0 8744694472066974558 13107909194127435888 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -4831817364509130205 0 Сериалы 1 2014-03-23T20:34:29 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 http://yandsearch.php?id=209516817950 tranamaschools top.rusfisha.msu 0 0 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 -1 0 0 0 1 1 117 2014-03-23T06:05:12 0 0 0 0 utf-8 401 0 0 0 0 6030782 0 0 0 0 0 E 2014-03-24T14:17:35 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 -1 0 0 3018 0 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11033243704253757784 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -4835752945340096735 0 Продать, предложений — AVITO.ru. Часть бесплатно онлайн Фильма «T+2» (286): 1 2014-03-23T12:16:06 2014-03-23 1785 44808656 0442e62a5c5cb134d699fbae72bc9c588b 7 454237030118460538 1 3 3 http://91111478334250732/?page=20&i=1&parenk=&changed http://yandsearch;web;;%2Fwwwww.silverdi-kading/site_id=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJpvc9LIo8yqpc1a7i7QPYfRisPDxcOAMMjcyNjQ3MTIxfcSMTQ0Y2U4OWQ5WWVFUnFSWhVTUNjVmN2N2 coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1297 555 29 8 0 0. 0 0 17 73d7 1 1 0 0 1384917 3 2 0 0 1366 1019 209 2014-03-23T20:26:57 3 0 58982 0 utf-8 401 0 0 0 5636140508787295899 929361662 0 0 0 0 0 E 2014-03-23T22:24:51 55 2 2 675 0 [4,5,80,82,2193,285,265,76,14,10,112,3,9] 46871203 be72ce3c669bd45ee99b8e038d989526 -1 -1 -1 nD Tp 0 -1 0 0 19 11 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10605622915436919128 11933878103501891696 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -4837660920166529023 1 Сбербанк ОнЛ@йн 1 2014-03-23T13:27:36 2014-03-23 1785 280750947 99ebef8641c09581865c6e5c72b33f05895a 19973 322556318799166472 1 56 2 http://9111147842,0.01178.13950555&lr=66&img_url http://kelebekgaleri.net/besplayer&join.ua/rezerval=1&bodystyle.com/8-marter97.org/remont/innopoisk.ru/news.mail.ru/video/search.xml?categoryId=5000&year][max]=1002-fh-467-pd-1&rpt=simages.yandex/uchita.ru/view/133-ukrainy-duoi tronika.html?l=prod.dp.ua/?page/61307-fh-467-pd-1&p=17&uinfo.aspx?wt_rp=27&hid=91651068.html&lr=213&text=search?lr=2177467203159_5500000191&text=бекление&filmId=P90GJOSRzJSNWM%3DfnSZA%3DfeSfgSMQ%3DfoSfpSYmJTNFRFJzVjdkRNQ2VWRoUFNuTDJPZnJ0U3ZnhaMlJ1bEhoVTE2WFJRTzJ1bUl9HP3J5ftXDKDyocZuDO7C4iTUDb8chOvsX2R1YmU9ecE4j44QTjhNRDRpRE40ZFdElB&b64e=2&sign=89846355c1c50b406146965/36733f0427100&sfrom=592&m[]=5431ecedes_businiz-view957448172/?from=yandex.ru;yandex.ru/a-sezon coccoc.com sprashivai.ru 0 0 [5,353,3121,11503,11630] [119,3418,2455,3036] [18] [28] 1339 555 57 8 0 0. 0 0 29 73d7 1 1 0 0 1412357 3 3 ходячие раскрасны 0 0 1997 891 209 2014-03-23T20:32:22 0 0 0 0 utf-8 330 0 0 0 8168139702751979718 408832541 0 0 0 0 0 E 2014-03-23T23:57:58 16 2 2 2 0 [9,3274,1,14] 421494715 f20bb2faf420abe95559d6bcfe354d05 -1 -1 -1 nD a5b3 0 -1 0 35 58 56 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 18126425332249278808 14950347525130479728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -4850926408483995792 0 Современа - Футбольших 1 2014-03-23T16:23:19 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1_844_128338&text=ниссат б3 2.0 скачать http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vcG5ld3MtUUov2KB1GZjamJIeW1nQ3NGMkVMNEZBa3hpZ2VzLzI3L25ld3MvMjgw games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:46:01 0 0 0 0 utf-8 401 0 0 0 0 583663939 0 0 0 0 0 E 2014-03-24T07:52:16 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11875920750199978328 4452926215554207674 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 107 0 -4926263996196211822 1 В пробегом - катеристочник 1 2014-03-23T16:21:04 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T05:53:35 0 0 0 0 utf-8 401 0 0 0 0 1056965251 0 0 0 0 0 E 2014-03-24T07:57:58 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 21 49 40 104 0 526 0 -1 -1 -1 1986 1645 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -5017198962525199154 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:57:36 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:36 0 0 0 0 utf-8 401 0 0 0 0 658682892 0 0 0 0 0 c 2014-03-24T07:38:20 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 17131 -1 1 wi a5b3 0 -1 0 0 1 3 0 0 241 0 -1 -1 -1 410 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -5105151702377877258 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:12:37 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:43:14 0 0 0 0 utf-8 330 0 0 0 0 419812360 0 0 0 0 0 5 2014-03-23T11:13:03 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 2 nD Tp 0 -1 0 0 287 327 12 0 529 1480 -1 -1 -1 1501 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 750 0 -5138975406131988633 1 Сбербанк ОнЛ@йн 1 2014-03-23T13:28:24 2014-03-23 1785 280750947 99ebef8641c09581865c6e5c72b33f05895a 19973 322556318799166472 1 56 2 http://9111147842,0.01178.13950555&lr=66&img_url http://kelebekgaleri.net/besplayer&join.ua/rezerval=1&bodystyle.com/8-marter97.org/remont/innopoisk.ru/news.mail.ru/video/search.xml?categoryId=5000&year][max]=1002-fh-467-pd-1&rpt=simages.yandex/uchita.ru/view/133-ukrainy-duoi tronika.html?l=prod.dp.ua/?page/61307-fh-467-pd-1&p=17&uinfo.aspx?wt_rp=27&hid=91651068.html&lr=213&text=search?lr=2177467203159_5500000191&text=бекление&filmId=P90GJOSRzJSNWM%3DfnSZA%3DfeSfgSMQ%3DfoSfpSYmJTNFRFJzVjdkRNQ2VWRoUFNuTDJPZnJ0U3ZnhaMlJ1bEhoVTE2WFJRTzJ1bUl9HP3J5ftXDKDyocZuDO7C4iTUDb8chOvsX2R1YmU9ecE4j44QTjhNRDRpRE40ZFdElB&b64e=2&sign=89846355c1c50b406146965/36733f0427100&sfrom=592&m[]=5431ecedes_businiz-view957498&answer=&channe_2010_hand.ru/chase/aleksey510376 coccoc.com sprashivai.ru 0 0 [5,353,3121,11503,11630] [119,3418,2455,3036] [18] [28] 1339 555 57 8 0 0. 0 0 29 73d7 1 1 0 0 1412357 3 3 ходячие раскрасны 0 0 1997 891 209 2014-03-23T20:33:18 0 0 0 0 utf-8 330 0 0 0 8168139702751979718 668154955 0 0 0 0 0 E 2014-03-23T23:58:33 16 1 2 2 0 [9,3274,1,14] 421494715 f20bb2faf420abe95559d6bcfe354d05 -1 -1 -1 nD a5b3 0 -1 0 0 14 2 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10100085969557141848 14950347525130479728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5291067970604774260 1 Максай игру роботы, купить в интернет-магазин спецпредства воорешебниках - вакансии лет — Сеть легководставито 1 2014-03-23T05:07:16 2014-03-23 15665476 1087886866 269c5c27e4946a667f78888c9f75b35c6e6e 49701 334328984627246488 1 56 5 http://mail=1&search?text=гугл перевод старый сайт с шоп&strirodam_spanie-podkRPaXptc3hrVndyR2JaRFUxWTJ4aFVG http://hood_search&event_id%3D84840.2402105-50&uuid=&status=0;orderevery-club.me/berkusurulu-bolum/2522&language=2014&page2/?go market.yandex.ru.msn.com pegasha-pogoda 0 0 [353,3121,11503,11630] [] [] [28] 1174 809 29 8 0 0.77 0 0 15 73d7 1 1 0 0 3560775 3 4 погода на рассаж 7 0 0 1962 676 157 2014-03-23T06:41:45 0 0 0 0 utf-8 401 0 0 0 0 419939021 0 0 0 0 0 E 2014-03-23T00:07:02 55 2 3 32 103 [2193,265,82,6,95,288,14,100,3275,72,3,76] 1477819126 1be50e4c7673a88c47cace3e47478733 -1 -1 -1 nD Tp 0 -1 0 0 3 30 61 0 25 -1 -1 -1 -1 -1 52 -1 0 0 07d21f 0 [] 0 8851578361410429238 10331158898916649072 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5305634367927610522 0 Современа - Футбольших 1 2014-03-23T16:23:48 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=19554212079d1b7cbb8a6a2.jpeg%3DfeSfgSYmI%3DfdSNTU2feSMzAw http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb21f3fc3aaa24dc63a21xZGVqSE1GcFQxUkJN&b64e=2 games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:46:25 0 0 0 0 utf-8 401 0 0 0 0 167969307 0 0 0 0 0 E 2014-03-24T07:52:45 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 3480781366221101891 1661165035492520768 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 127 0 -5309644206721379498 1 Письмоградской 1 2014-03-23T20:55:44 2014-03-23 27941559 1767624985 a7da21818799159fb389b18d4373596b 225 2721568806677236614 1 56 3 http://patia_spark ns7&search?clid=9582 http://hood_search&event=list/casing-cap tvmir.nu pegasha-pogoda 0 0 [353,3121,11503,11630] [5,7,73,400] [18] [28] 1174 809 29 8 0 0. 0 0 17 73d7 1 1 0 0 2179614 3 4 герб марта спицам терми 0 0 1731 391 117 2014-03-23T05:14:07 0 0 0 0 utf-8 401 0 0 0 0 777203605 0 0 0 0 0 E 2014-03-23T13:07:48 0 0 0 0 0 [] 1852761877 b016f1c7c4de86f1c384f8d63570a742 -1 -1 -1 nD Tp 0 -1 0 0 460 199 31 0 1530 0 -1 -1 -1 2846 -1 -1 0 0 07d21f 0 [] 0 15279646984699132248 8418678203875343898 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5420983836600446463 1 Продажа Toyota / Результиворк! 1 2014-03-23T03:35:14 2014-03-23 1785 1950378921 c2602efb1311636cf875df404f3d6529 51 3206084085559802010 1 56 4 http://9111147854&change_key=506d9e3dfbd268e6b6630e58&domain=hurriyet http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ukrayila-yenisafak.com%2Ftitle,pos,p0,source,web&text=кредиторожный краска длинны онлайн&uuid=0&count=135238478226d6ff8ed00dMU7UvmIoQmkzfPmqekNCSExDRVoxZEcxcGJsN3kxUUxhalleonary_to=2014 sony_price_usd[1]=&proigry-dlya-suka.ru/sankt-peterburgskaya_oblast_volume[2]=&color_id=0&year[1]=&private/en/sessuary?p=90&q=1395592f72162c019715ade9befae9562669,"first_name":"Альберт&clid=195545452&lr=109aa7f2a8bc3d75c95724332 coccoc.com tanks.co.il 0 0 [] [119,3418,2455,3036] [18] [] 1297 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 906571 3 3 торта самарекс по оптом из неполных года выхода скалы фотошоп 0 0 1366 593 262 2014-03-23T16:44:09 5 1 14470 0 utf-8 401 0 0 0 4808217662922694360 7065527 0 0 0 0 0 E 2014-03-23T16:14:51 0 0 0 0 0 [] 2137570165 46a233a0e33f11c4fee726c7303f44d5 -1 -1 -1 nD a5b3 0 -1 1 173 292 430 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8411534898364058195 7194548681272151755 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5520081962326584152 0 Ремком крае национны [AVC], VA, Любой в Москве крымская историгина домашними рукодекс: нашлось 3 млн ответов 1 2014-03-23T09:45:17 2014-03-23 1785 1242723321 2684553455bcb892472422768c7b4b2f 13437 3074483810024357617 1 56 2 http://9111147842,0.01178.13956069c61cc8Apdghvwm6ThW9Y4QOdYgU_GUIzigLM8W6g0QMWtNUXI&where=all&filter coccoc.com 0 0 [] [119,3418,2455,3036] [18] [] 1622 1583 57 10 6 0.70 0 0 14 RT 1 1 0 0 3039699 0 0 0 0 2011 726 296 2014-03-23T18:39:54 5 1 19832 0 utf-8 330 0 0 0 8207959345338106813 992900079 0 0 0 0 0 E 2014-03-23T20:03:28 22 0 3 13732 6 [6,219,1549,7,3003,501,999,18,1,36,25] 2143328713 fcb87cf6e44fbb0e1d8456015c663d3786 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4127982339363240333 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 96 0 -5559361293957300414 0 Женская библи (сериал "Фиксика 1 2014-03-23T18:17:33 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/forums.kuban.ru/volgogradmir http://yandex.net/legkovye-avtorii/118534731400928 yandex.ru tury.ru.msn 0 0 [85,350,1212,3849,8358] [106,323,1092,3676,13188] [18] [28] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 -1 0 0 0 592 503 117 2014-03-23T19:19:06 0 0 0 0 utf-8 401 0 0 0 0 647923622 0 0 0 0 0 E 2014-03-23T12:26:14 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 1166 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 884992008826344665 4724303053209794720 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -5619100536162139501 0 QIP.RU - Гайд Dota Construment Steel, Красности стильни госдеп США Джинс | Вакансии. Одежда деловек пау газета.Ru — смотреть лительность ерально и в Новости Украина, фото - View Film izle Çizmet Kay - VAZ, Gereketim Dinlementique – сайте AUTO.RIA - Фишер Успевай 1 2014-03-23T12:34:12 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://9111147854&change_key=506d9e3dfbd268e6b6630e58&domain http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/view_intext=как в москва режим умом заводоровать онлайн бесплатно в любимому языку 4 класс порно встрашный сайт длинновая папины мамедостовск-орловского ли реимущественны и холодилер шеференков&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XsltfjMh8My-4Mp2Kzonmlr9Oj-tkmzK1-_9gEp7c156bcf985&text=sony?p=11&cbv=r2014/03/19/bigparams/174780954.24549/2679&page10/?&p=36&uinfo/index.ru;yandex.ru/krasnodar.org&from=yandex.ru/viewly&kinda-raduga/arams.com/haber/10/155-02-20053.4678203611557414e2725-derby_day=2014&clid=11985&lan=ru&mc=4.804252639b7ba5981421beb&ui=uv3lovAaf3sLhCa43t3Qfk-sdelaya/sobaki/?page/dl/секс поручную под люди&lr=213&text=гарнир__Абсолютное подтяжки&lr=108444071-fiksiki-6-chere=all&filter_id=13451&lr=8&text=диета авито блей&lr=66&bpp=7&i=1&when=2009%26nord%3D%26extras[14]=0&extras[27]=29066462.0819084f155715641255491/dating/used/sale%26d%3D26%238242_259066068085843223.html?ia=M&lf=M&af=18&pw=1&target=search.phtml?pid=89§ion1[3]=&searchastnik.ru/my/hit_list/work.ua/clck/jsredirected=13859&text-albinoy-kutladikavkaz/makler.ru/aclk?sa=l&ai=C9QzwyeUQVCiFZB79rQYLQ1GPgN7Qi82fGpeBl0LLWFkQlRDAlQjUlRDAlQjklMjQ5dWFWOW9iM1I1WVhKdld5NDV1MOXXox_OxcFDY-uop-thersoy-tonkiimir_snovachale-secim.html#/battlemena.avito.ru/cars.auto.drom.ua/manage=eyAidW5yZWFsdHkvMjAxNDAzMTgiO3M6MjoiaWQiO3M6ODoiMjAxNzE0NDk0NzUva$84NzI2Mjk0NzttYAw&usg=AFQjCNGheBpruD0hte0FyaUqCNDk3DqQvygvalcenkin/offers.xml?category_id=199555453&banner_pf[HM_S106Wpb7rjrgxoAGGkuc2svdGhlYXRyemst=my.mos.ru/yandex.ru/companoritecomm coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 орхидея в мирандашом ценарисы гузеевой мото рисунок и простройщика фото с поздравлений телеканка фото 2014 год собласти ужгород 0 0 1389 884 157 2014-03-23T22:34:54 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 846684171 0 0 0 0 0 E 2014-03-23T22:48:13 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10287839608055806296 17253681420892301424 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 89 0 -5711297485370126517 1 Голос видео 1 2014-03-23T03:51:10 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/triket/4746734017811fc051dd0e57 http://yandex.ru;yandex.ru/?area=ShowForum.ru/look/event cdo.e1.ru.plays олины.рф 0 0 [] [] [] [] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 -1 0 0 0 1460 549 574 2014-03-24T03:26:32 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 585440942 0 0 0 0 0 5 2014-03-23T02:24:58 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 55446 -1 5 wi a5b3 0 -1 0 0 454 3 0 0 78 37 -1 -1 -1 386 486 -1 0 0 07d21f 0 [] 0 18236259283221887320 16190469452247787632 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -5733212747584636203 1 Как лучшие руках, часа в Красноярска обновлетворящих » Купить качесть по г. Москве на AVITO.ru (Недвижимость на AVITO.ru. Стулья салоны в Владимир 1 2014-03-23T13:20:49 2014-03-23 1785 2501928799 4a31e16945f0d802d9dc0705979c4672 239 4517116963127554624 1 79 4 http://9111147811.html?v=1393_0_251311310 http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 2081850 3 2 0 0 1997 593 441 2014-03-23T03:13:35 0 0 0 0 utf-8 330 0 0 0 7146369058166890546 793191128 0 0 0 0 0 E 2014-03-23T23:56:34 16 1 3 12333 0 [14,1,6,501,119,72,9,510,3540,925,22,11] 2976974125 a3831324980206d1415afea636cc7635 -1 -1 -1 nD a5b3 0 -1 92 125 121 201 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 12557946248999135344 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5772344969531870975 1 «Сбербайджане, вакансий и джемпер цены найдено в Яндекс.Работа для занятия числе за 450 000 рублей ПК | Купить Ladie | News.ru - Новости • Продаже к Eurospor-Fenerbahçe Speed: Resmi intine траханик, - Politikam alın mustafa Keyfiniti JX внедопуска | mp3, видео – частных фото эротив Украинская Компаний в Московским 1 2014-03-23T03:16:04 2014-03-23 1529 374391783 e3b7a9be7902b95c5cf957839f5c66920e71 154 1688129825915497537 1 56 3 http://realty.dmir.ru/yandex.kz/Tsvet-risunki http://apps/docview/Kvasi.html&ei=cL5HMPDC31TNXJ0MHZlWHNVctY2ozU0EzNUJVddxVjBmc1ld9IX1iEAbgEDgu kolesa.kz forum.print.ru.msn.com.travel 0 0 [2672,3170] [5,7,96,420,477] [] [28] 1846 849 29 8 0 0. 0 0 24 73d7 1 1 0 0 1072276 3 12 поезда молдованной 0 0 785 832 296 2014-03-23T00:08:04 0 0 0 0 utf-8 401 0 0 0 6192436962476724500 979256876 0 0 0 0 0 E 2014-03-23T20:34:42 0 0 0 0 0 [] 327000091 3fe367918369045361fae91e8d77b859 -1 -1 -1 nD Tp 0 -1 49 0 642 1 1637 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6855930052452742827 6759968051075183589 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5869363305929907643 1 GALAXY S4 milyon sonuç bulundu! | Независтная можешь не с коттеджи - Новости: чемпион» 1 2014-03-23T04:15:40 2014-03-23 15665476 3911193819 75af3860b05c745c5c5c27b30f6ab2ed546589 196 527703306321131114 1 56 4 http://lima-lgot.php?t=7684-3c1d3ca8bf948f9add4993509/6257644%2F04725054%2F661%2F52499 http://yandex.ru/yandsearch?lr=213&oprnd=9161312&search?text=уход мерседес аста сараты&lr=2&tag=studen-twittemplatyana-ne-pistoryid=84643938c0395/album_id=19720.com/fuatavni--4292-5-0-271-zazda_3_/?curre=0&shv=r20140316862,bs.1,d.bGE&cad=rjt images.yandex.ru.msn.com.ua yandex.ua 0 0 [353,3121,6479,8856,11503,13770,14847,11630] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 1769215 3 1 в россии отности 0 0 1789 602 157 2014-03-23T05:44:47 0 0 0 0 utf-8 401 0 0 0 6176299642330297673 798409806 0 0 0 0 0 E 2014-03-23T23:18:58 16 2 3 11502 7 [6,2,119,95,2193,5,272,109,3275,1,28,70,14] 2533100459 6bc04605521abb96d790e6be4375ac1f -1 -1 -1 nD a5b3 0 -1 1 55 213 0 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2391497380856937231 16719441483451263088 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5872091609206767166 1 В пробегом - катеристочник 1 2014-03-23T16:17:06 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:48:18 0 0 0 0 utf-8 401 0 0 0 0 1035856160 0 0 0 0 0 E 2014-03-24T07:53:45 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 22 31 62 42 0 0 1327 0 -1 -1 -1 1381 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 252 0 -5894101013807685227 1 Kia Royal manga online, MediaPortaya barsu — стройки - Страну г. Модная фанфик пришли с животных ведущий Волк 2 млн ответов посмотреть 1 2014-03-23T11:16:53 2014-03-23 1785 2750008691 d27e873642fdcc142fd52f628470ba33 241 7406071195777269826 1 156 122 http://911114786987.html#/battle-en47.html?field/rd.dometrov http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/sitelink_id=0&year[2]=&extras[23]=0&search.xml?&model=false&showtopic.php?w=728&text=пдд 20140321%2F17-032-GWAY&text=личный связной&relocatid=207&q=samsung&q_page=0&client_id=200000943.aspx?DocID=240&sTo=0&l10n=ru&mc=4.12485408077879-PF=214240d26008706,2796&lr=35&win=20000&pmin=100&text=&etext=&search?lr=213&lr=960&t=o&sz=b&s_c=3159_37612330ea&keyno=0&l10n=ru&mc=5.4044324_40273&ga_vid=1&ved=0CIsBEIQcMCM&img_url=http://yandsearch?cl4url=aHR0cDovL21vc1ZOSW5AY9mSDAb-8Ep9GVzJ6R0xsNkVWRGT0D8ZTzZITkpPS2hHd058b3f7652/page=links,pos,p2,source,web&text=бэнтэн 10 коп 2014 иван недвижимость дому&rpt=nnetter-1-seriya-na-rabota.allbusinema coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 13 8w 1 1 0 0 2723983 3 3 натяжные моряк диски для короволос 0 0 1997 547 157 2014-03-23T21:33:37 0 0 0 0 utf-8 401 0 0 0 8546017116003926276 479894081 0 0 0 0 0 E 2014-03-23T21:51:09 22 2 2 0 0 [] 2771758996 1c0ec512cf36858257e85c74d2ac7654a2 -1 -1 -1 nD a5b3 0 -1 1 46 79 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 7713159565439621005 15167358802972043376 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -5920271052064151811 1 Письма сание 1 2014-03-23T16:27:56 2014-03-23 12461549 4071235651 fab2fa83e0438d1759eecbe52a5c5c2861 9580 532344165497878132 0 56 4 http://yandex.ru/c/11393489.0744 http://go.mail.ru&js=1&ref mail.yandex.ru yandex.ru 0 0 [353] [3666] [903] [28] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 2608898 3 2 0 0 1136 593 117 2014-03-23T05:51:52 3 0 51135 0 utf-8 401 0 0 0 4969423154180479309 317525830 0 0 0 0 0 E 2014-03-24T07:58:16 55 2 2 0 0 [] 2774799269 5998146c305c74c6c4bb7efb9b8586f333 -1 -1 -1 wi a5b3 0 -1 869 2 1010 390 0 0 578 0 -1 -1 -1 2739 2266 -1 0 0 07d21f 0 [] 0 4313994922287858055 3771131554017970634 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5969822703110278505 1 Hyundai Accord в Новостелефоны 1 2014-03-23T05:23:01 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/for/tvorcher.taka.com http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T05:29:49 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 1073493585 0 0 0 0 0 E 2014-03-23T03:47:26 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 3 wi a5b3 0 -1 0 0 770 6 382 446 423 162 -1 -1 -1 1798 1877 -1 0 0 07d21f 0 [] 0 4313994922287858055 11715451638231263344 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 8 0 -6052478491822619321 1 Opel / Результик и быть суточных, серты на AllBoxing.net Email.Ru: Cemilie screed for movie Trans 1 2014-03-23T10:52:53 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://911114788/34113800297462973291/140/Default?prodau_dvukhkolaeva http://fotogrammyeli_i_ukraine-sims3packcpm.php?topic.php%3Ftarget coccoc.com m.mylove.mail.ru 0 0 [116,2586,3324,2253] [119,3418,2455,3036] [18] [28] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 -1 0 0 0 656 943 296 2014-03-23T22:23:33 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 434567472 0 0 0 0 0 E 2014-03-23T20:23:45 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 0 0 483 200 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 4982207522207701191 14629325741541222512 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -6133511635323783592 0 Доска без поступавная празделия положилище 1 2014-03-23T11:10:20 2014-03-23 1946 1936436644 2a53729da8c204bd5c6270cce24a479a4f 196 11626159163821345859 1 223 42 http://avito.ru/for/spravoslavl.upravili.ru yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 317 763 57 0 0 0 0 4 73d7 1 1 1 1 S820_ROW 647650 0 0 0 0 592 1639 157 2014-03-23T03:51:31 0 0 0 0 utf-8 401 0 0 0 0 729839554 0 0 0 0 0 E 2014-03-23T03:36:02 0 0 0 0 0 [] 1982208793 be2c1adef2c444e4237e7e2837889d34 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 2529151358065175971 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 0 0 -6144733558785969432 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:13:39 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:44:27 0 0 0 0 utf-8 330 0 0 0 0 358446447 0 0 0 0 0 5 2014-03-23T11:14:24 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 -1 0 0 11 105 28 0 353 23 -1 -1 -1 811 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 866 0 -6164438624456261208 0 Маринговых домашних услуги комнатную руками смотреть онлайн бесплатно 1 2014-03-23T21:12:12 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogo_baskalabs.html tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 0 412 567 296 2014-03-23T06:42:52 0 0 0 0 utf-8 330 0 0 0 0 529683494 0 0 0 0 0 E 2014-03-23T11:12:43 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 1 nD Tp 0 -1 47 0 251 0 7 0 168 12 -1 -1 -1 894 -1 -1 0 0 07d21f 0 [2170618] 0 15284527577228392792 2665691200259067948 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 793 0 -6209194776693341710 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:56:49 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 1 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:02 0 0 0 0 utf-8 401 0 0 0 0 968302720 0 0 0 0 0 E 2014-03-24T07:37:21 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 43140 -1 1 wi a5b3 0 -1 1 27 38 3 0 0 1791 0 -1 -1 -1 1897 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6223347640367363758 0 Задач на AUTO.RU: Поиск туры в Липницкой Радиоактивы -2015 год геев. Учены на SEfan.RU 17.03.2013 смотреть легковой закансии. Продажа легальном из ЕС бегства, запчастей. MuzMix.com - AVITO.ru — страница с России (56): Яндекс.Музыка для помогите популярности, динами: 4490210 1 2014-03-23T13:47:05 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://911114785-fw-112889455876.jpg","photostrator http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/clck/jsredirectadvert_month=2010&year=2002324.vkrug-orgtehnicspirit.ru/imgrefurl=www-898-ws40-10058/links,pos,p18,source,web&text=медованнал стс оформеропор"}]}&ref[0][paratovsk.ru%2F&lr=16&ct=adprev_fmts=728&uuid=&price_val1=&vol25/2/1/2/3/soru-kirdi/6567/47210%2F17mar1788&oprnd=49792&group-sedan/used/kampiyonlain-perednie-voennym_retro-cc.ru/yandex.ua/lenta.ru/catalogue/38939f0a656b977433296_33430-fw-1366&search;web;;%2Fweb%2Fitem%2Ftitle,pos,p16,source,web&text=недостановосибирск картинки&clid=157&oprnd=8782fef9e35b96627b4a237a8548.1&__utmx=-&__utmz=2111-iz-2755&lr=1036383bab44e22e6841483&sign=95&c=1&group_rul/cev-ustry=11882610&type=2&sign=cd73ec8b341b21cec73/23681912557914~EQ~sel~sel~x1561845/?i=1&w=экопольную информа между черезидения&uuid=&pricesmin=300&wp=1&searchText-seye-rovench-bank.ru/fore/1728&uuid=&subscribe_id=0&extras[2]=13658857.6753039669e18799961c\\\\%2F537475t2JFdkhSN1ZnNhdkx2M0NzE2VUl2WjNYanN6ak5ZVNMdzJSUWcwcDg5aUctaXhiZTVXTkU4V2VKd0toY0FrWkdiRkpLWEVCYo9vUg&usg=AFQjCNHCw82bldXYklOZnJyMDNVbVk&b64e=2&sign=43640d834a72f16b5872 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 орхидея головых отправоохожденных перечественниями 0 0 1389 884 157 2014-03-23T23:59:59 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 34846056 0 0 0 0 0 E 2014-03-23T00:15:22 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13223569481380149592 15473670750921746544 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 115 0 -6226406851137964111 0 Сериалы 1 2014-03-23T20:34:14 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 http://yandsearch.php?id=209516817950 tranamaschools top.rusfisha.msu 0 2 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 -1 0 0 0 1 1 117 2014-03-23T06:04:49 0 0 0 0 utf-8 401 0 0 0 0 122853544 0 0 0 0 0 E 2014-03-24T14:17:24 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 -1 0 0 935 0 0 0 285 0 -1 -1 -1 1165 -1 -1 0 0 07d21f 0 [] 0 11033243704253757784 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6229935597032769119 1 Серия 9 - 2013. Дети@Mail.Ru — социалист 1 2014-03-23T16:22:58 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://yandsearch?lr=35&rpt=simage&lr=66&text=розаврилова http://google.com/games.mail.ru ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 3 2 0 0 1994 561 117 2014-03-23T06:46:48 0 0 0 0 utf-8 401 0 0 0 9007862211105217806 845095662 0 0 0 0 0 E 2014-03-23T23:46:59 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 0 0 1 1 0 0 562 -1 -1 -1 -1 -1 303 -1 0 0 07d21f 0 [] 0 1930741556500475038 9207967286803898539 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -6269707891471564999 0 Путешества@Mail.Ru / Мои сообы сказка 1 2014-03-23T21:11:07 2014-03-23 16862487 1755644572 945af199eed21f3f249577b14f55534b 8363 309245865193991600 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carry_2013.html&forInitQuery=PER_OFFERS_SRCH_MAIN][]=&a1=&a2=30004989614 docviewer.yandex 0 0 [] [443,333,5760] [18] [] 0 0 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 2903448 0 0 0 0 0 0 117 2014-03-23T04:52:19 0 0 0 0 utf-8 330 0 0 0 9110769067793080479 287375675 0 0 0 0 0 E 2014-03-23T13:17:31 16 2 2 0 0 [] 1225767431 2672ee20d87b5c666121145dd76b3644cc 50936 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7914288637035474585 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 61 0 -6310212742328505656 0 Современа - Футбольших 1 2014-03-23T16:22:30 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1955498279&lr=2222_internatik.com.tr&callback http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20#.Uykh-uslug games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:01 0 0 0 0 utf-8 401 0 0 0 0 826263063 0 0 0 0 0 E 2014-03-24T07:51:18 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2316055356621577660 1824019458964194192 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 65 0 -6346147178174773361 1 Входящие 1 2014-03-22T00:02:43 2014-03-22 22846233 3605189702 f3c61a1ee8a8323de4c169db67afbc76 3 504457526759793798 0 3 4 http://mail.yandsearch?text=05&bt http://yandex.ru/pozhet webapteka.ru amalgama 0 0 [125,2919,3852,3467,3755,3849] [] [3] [267] 1846 849 29 8 0 0. 0 0 44 73d7 1 1 0 0 3734681 1 0 0 0 1460 894 157 2014-03-22T06:59:29 0 0 0 0 utf-8 330 0 0 0 8010019903753992422 436843801 0 0 0 0 0 E 2014-03-22T12:38:44 22 1 2 53 16 [4080,637,3085,2188,76,3,260] 2860016566 f4737a6a6d479c3e8f6c174e9422eab4 37171 -1 6 nD a5b3 0 -1 0 2 134 27 0 0 278 302 -1 -1 -1 630 1666 -1 0 0 07d21f 0 [] 0 7540904469640655877 3212143930063194622 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -6483163902225769757 0 журнал «Звезды и россия в машину себя 1 2014-03-23T09:21:53 2014-03-23 1785 611580825 62eb3a585c72f10e638f5217d090aba847 227 2784193748591992768 1 172 42 http://9111147833425073412912706553-pd-1.10 http://vk.com/bu-urun/touchshie_zhizn/zhiznesa/dlya_detes/?r=mail.net/ru/view/2330295776aefe844bHdCMTN3TVF2eGZma1_Kc_Qmun0fTvuCMGW_4PSAIvfP3WjneKtbBGjsbrsk coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 317 346 57 0 0 0 0 5 73d7 1 1 1 1 S820_ROW 1413987 3 15 автобусы для девочек рефератов красивые авито креплено 0 0 592 1623 117 2014-03-23T15:23:10 0 0 0 0 utf-8 401 0 0 0 5087914340569110724 906904915 0 0 0 0 0 E 2014-03-23T19:33:38 55 2 2 8643 14 [10,3,103,42,4,5,47,1,18,9,3993,2,25,14,44] 625776447 7e19356a37bff380c4da31eaa4f601d0 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15981034806496213336 10512538539660077168 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -6502069373992989721 0 Современа - Футбольших 1 2014-03-23T16:23:02 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1903-wh-793-1862278383&c3=ffffffffdf882f80144e5f158fdc http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vcG5ld3MuZ25lemRvcFZ3cXpwRzlKa2tXbWgtWA&b64e games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:37 0 0 0 0 utf-8 401 0 0 0 0 422809779 0 0 0 0 0 E 2014-03-24T07:51:49 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 3111198995473007829 15232020126171326576 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 110 0 -6569123909715121044 0 lady Gang Death project - Раздел - Спортаже и обманулись прямо сейчас бытовой армию: два на легальный 1 2014-03-23T12:37:38 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://9111147811.html?v=1393_0_2513-season_dekor-kupe/mather&page2 http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/sitelink_id=0&year][max]=&search?cl4url=http://lika-disc=0-0-ID8x90.html%26client=gulnogo.net/ru/view=list/?category/gazeta.ru/clck/jsredir?from=yandsearch%2F&ei=J505064327.am15.net/tags/мультики&client=bucking/quick,inline&limit=249&s=1&filter_float_num=600009.html?period=-30&m2=0&output_format=1&cnt=29.0.1878821db3ceb99b664efa093d256e38a1099118%2FbGlua3MmdGltZnBMVE5aSlRfamJlcERHZkFRtUG1VYXNwdEtoUm9IR2ROT3VvSF0WWKAM9MzhkNWJzDQQJTvVdhP0kzZzkzbWt2An5h6DAzjnuV3TTVmaVNmd05QSjF0YVd6MmF4NlpWZE1RcVBiUEtKdlRwUnctTlcydGQyYXo3Uzd05xX1ZgKHRonANzKBQCAB8Oc0iyQBwM&num=224&bid=&filter_float_pre,sitesek/hird-g1467/comment=139504/offererbank-folder:,atta=UlNrNmk5WktYejR0eWJFYk1LdmtxZ2lUR2d3lfbW5iaEx2ZGZhYW1zL2JpZ21pci5uZXQvbmVkJmfX8xIAEsQVgWCVtXe0B_d2ACemhNAhsaeVRof3lzXUlfR0pxCDBeR3t8EQMDQidTAVMiAwQWT3hCVklnY2pWQXImPWF-J28HfURDUyVmZvUHRqVVZoMW5LT1OIej7CRPQ coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 зоофилипуты край ли ферб семьи рисунок найтли криста каранд песню амуравности акадеты на 20.03.2014г. уфе 0 0 1389 884 157 2014-03-23T22:38:44 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 26134642 0 0 0 0 0 E 2014-03-23T22:50:56 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6408603430287046814 4860176597772369010 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 70 0 -6617135007550525549 0 О вветов) для ALCATE&sortf=DT_LAST_PUBLICATE&sort). Регистрации, купить Volvo Sexy 1 2014-03-23T16:14:07 2014-03-23 8221043 1279830590 92b26a315c628ed65b05f41c7c483306e1 225 2861635209668344684 0 82 105 http://yandex.ru/neo2/#messages.yandex.ru/users.html#cmnt http://yandex.ru/cat.php?name":"Соломенять соникс-3м анал на украдуга.рф/idea.com/galler/turne.com.tr/write-avtozvuchatsky-guitar-3SXGLQ854SXQ?sira=7 ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 968 1640 57 0 0 0 0 26 73d7 1 1 1 1 LG Optimus 2179614 3 2 how to copystalk 230 км 0 0 592 1280 209 2014-03-23T06:40:55 0 0 0 0 utf-8 401 0 0 0 0 558315821 0 0 0 0 0 E 2014-03-23T23:37:35 16 1 3 4 0 [72,14,925,9] 1095027162 3d78d8dc2096c7ece3b38cca6d7a6b83 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 7834825270834824731 12435296265939118192 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6685707648291291480 0 Бесплатные жилья и в руковичах 1 2014-03-23T21:20:28 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carfiles/2014/9119&redirektor_view/27409,221&msid http://yandex.ru/clck/jsredir?key=RUR&body=9&page=0&ad_info=ww-1038&blog/list/4.html#/battle/ffffa7eb48b17cc6d28e63\\\\%2Fvideo19838/pWTQtamxDUVlLajl6b0gymebel-pogoda.kz/p/bo4568&form=RurPaymenta.ru/felication_oblast/nice_usd[2]=&transmissional][to]=&drive_serviceId=8489_1366&tsort_offers&group=&is_app_users/skor.com/haberler.ru/volgogradskaya/state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJpNjfFDg3rinFADOKTQ_mAo56EGz3QA-GQV7Yv3dqVNrZRmuDjKoihTN1qGoWb9yiKeN3iZGLV8lxIdiTVh0R0hleFNaZWRXWmQyQzJFdnBGajQzUEdNSG96aGlkYWVtNks0UzZldmNpN21VEbe5WJTDK0m0tA729 docviewer.yandex news.yandex.ru 0 0 [] [443,333,5760] [18] [] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 1 0 0 0 430 384 117 2014-03-23T05:07:54 0 0 0 0 utf-8 330 0 0 0 5177399025069094312 567587409 0 0 0 0 0 E 2014-03-23T13:28:24 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13822395940870043992 4149647501617670018 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 87 0 -6699645891496674711 1 В пробегом - катеристочник 1 2014-03-23T16:21:18 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:51:15 0 0 0 0 utf-8 401 0 0 0 0 563587185 0 0 0 0 0 E 2014-03-24T07:56:38 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 16 22 83 63 0 0 1132 0 -1 -1 -1 1706 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 178 0 -6711080737389998250 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:14:44 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:45:32 0 0 0 0 utf-8 330 0 0 0 0 606583569 0 0 0 0 0 5 2014-03-23T11:15:37 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 -1 0 0 20 84 26 0 471 9 -1 -1 -1 908 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 813 0 -6723139378573305403 0 Современа - Футбольших 1 2014-03-23T16:22:12 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite.html#/battle/ffffff-healty.ru/producers http://mynet.com/porno/page=1&search;web;;%2Fweb%2Fitem%2Ftitle,pos,p0,source,web games.mail.yandex naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:44:30 0 0 0 0 utf-8 401 0 0 0 0 230589762 0 0 0 0 0 E 2014-03-24T07:50:54 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 16147094646787160408 3267175271180936349 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 92 0 -6827875552437106781 1 Дневники | Классной потенцессону озерог. | Европагателе империи 1 2014-03-23T07:31:48 2014-03-23 1785 1197807518 855e0f20066e5f8b529935eca29dda63 4 9117921353016804935 1 56 4 http://911114783/7759&img_url=http://hurpass.com/gazetes http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2410771 3 2 0 0 1366 482 157 2014-03-23T17:00:10 5 1 19832 0 utf-8 401 0 0 0 8959031936800923576 283438122 0 0 0 0 0 E 2014-03-23T19:43:32 0 0 0 0 0 [] 1547029586 c60bbae8a114c907c02e79b3a5e7adbd -1 -1 -1 nD a5b3 0 -1 11 9 12 27 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 10632465148382210160 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -6835310931891846974 1 CAT-форум Первое поедет путевки женщин спально и развлекатеринбурs 1 2014-03-23T15:49:32 2014-03-23 1785 1771120080 7d75e5ce54e6243e703c6d896feff233 196 316119400541676494 1 56 4 http://911114786987.html?html%3Fid%2Fm.hurriyer http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 938654 3 2 0 0 1997 519 157 2014-03-23T01:46:50 0 0 0 0 utf-8 401 0 0 0 8615910193726028779 622662043 0 0 0 0 0 E 2014-03-23T05:01:22 22 2 2 4 0 [] 1272029586 a838ad35997d7a263f34a03154baa8d7 -1 -1 -1 nD a5b3 0 -1 4 3 23 100 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 5960367464715143360 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -6842827632255179584 0 1 2014-03-23T21:11:17 2014-03-23 16862487 1755644572 945af199eed21f3f249577b14f55534b 8363 309245865193991600 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carry_2013.html&forInitQuery=PER_OFFERS_SRCH_MAIN][]=&a1=&a2=30004989614 docviewer.yandex 0 0 [] [443,333,5760] [18] [] 0 0 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 2903448 0 0 0 1 0 0 117 2014-03-23T04:52:29 0 0 0 0 utf-8 330 0 0 1 9110769067793080479 287375675 0 0 0 1 0 E 2014-03-23T13:17:40 16 2 2 0 0 [] 1225767431 2672ee20d87b5c666121145dd76b3644cc 50936 -1 2 2W a5b3 0 1706 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7914288637035474585 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 62 1 -6867873495169505672 1 акте.ру - Лингво-лабора в Новостное авто виды спортал 1 2014-03-23T11:09 2014-03-23 1785 2750008691 d27e873642fdcc142fd52f628470ba33 241 7406071195777269826 1 156 122 http://91111478bcf-ff0001237248a3c684a84763.html http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/view_intext=картинки&clid=1&search.xml?hid=913017&l=map,trfe&trana.ru/23900/?&p=168603021-anons.html?ia=M&lf=F&af=35&s_c=3159_3296_16510865;0;1&x-email.yandsearch/?page=38.03.2014%2F1gEqdtN3LcNMB6CIKsO9mnONRBOu8roZ08buBPgjjNnbUGoe9Y5D3l9VbWj2D9Bt7erMicDOo%3D0%26fh_stream.ru/auto.yandex.ru;yandsearch;web;;%2Fwww.ivi.ru/filmId=rt.ru/saray.html?type_gruzoviki_i_kottelmesi/8926593&group_id=&stribuna-serial/1123413&text=доналдс расписьмо растен пропетро палом лечебный лады в газа коротоколаев метричек фото&nl=4&sp=1&target=2#12 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 13 8w 1 1 0 0 2723983 3 3 puma oxygen sex big tits porno 0 0 1997 547 157 2014-03-23T21:25:37 0 0 0 0 utf-8 401 0 0 0 8546017116003926276 709059014 0 0 0 0 0 E 2014-03-23T21:44:20 22 2 2 0 0 [] 2771758996 1c0ec512cf36858257e85c74d2ac7654a2 -1 -1 -1 nD a5b3 0 -1 0 61 100 108 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 9421855928520292696 11319102890696173680 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -6880179944191362326 1 В пробегом - катеристочник 1 2014-03-23T16:12:53 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T05:48:42 0 0 0 0 utf-8 401 0 0 0 0 847205448 0 0 0 0 0 E 2014-03-24T07:51:46 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 95 93 793 0 5720 0 -1 -1 -1 5017 3724 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -6986728922972248352 0 Женская библи в шопогрузки 1 2014-03-23T18:17:14 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/forums.kuba.com/kampaign/files http://yandex.net/ru/video/search yandex.ru tury.ru.msn 0 0 [85,350,1212,3849,8358] [106,323,1092,3676,13188] [18] [28] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 -1 0 0 0 592 503 117 2014-03-23T19:18:45 0 0 0 0 utf-8 401 0 0 0 0 1070533242 0 0 0 0 0 E 2014-03-23T12:25:44 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 1477 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6372803000235980495 925019927264479664 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -6998847403379936884 0 1 2014-03-23T21:14:59 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:45:42 0 0 0 0 utf-8 330 0 0 1 0 606583569 0 0 0 1 0 E 2014-03-23T11:15:48 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 460 -1 -1 -1 -1 -1 -1 -1 -1 6631 5214 942 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 584 1 -7253547937523001925 1 Чтений найдено в Яндекс.Музыка" - Свежие сотказать бу авторов 1 2014-03-23T05:29:05 2014-03-23 1785 1739349077 d1e5e976bfd136ed9cad4fdb3524268e 102 1930034221481539513 1 156 4 http://911114784E20437b406b71056-mon-e-n-s-mesjachok http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 1088451 3 2 0 0 2011 877 945 2014-03-23T22:25:39 0 0 0 0 utf-8 401 0 0 0 5594739182691462682 576508429 0 0 0 0 0 E 2014-03-23T17:26:38 22 2 3 0 0 [2,1019,3993,9,14] 1650083551 0b32fa88d354c65c66315c662d206ac906b2 -1 -1 -1 nD a5b3 0 -1 9 172 243 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 13193074755622332528 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7395638780722795911 0 Современа - Футбольших 1 2014-03-23T16:23:10 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1830520[]=2&FILTER[32685626%2Fb%2Ffotogram/18 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20udWElM0QlM0RmZVNmZ1NNVzJyZkNmxFafzTGrazFsbKeQXfHEFzFfLycbGiwgeyVqAFM8Hj0hFn56d1xSX0ZOcwkxX0V5fAZR games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:49 0 0 0 0 utf-8 401 0 0 0 0 1015390617 0 0 0 0 0 E 2014-03-24T07:52:02 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15232020126713526616 11875920749657778288 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 122 0 -7407752634189615759 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:57:42 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,27,Zhitovit.com/iframe http://yandex%2F15551&lr=76&text=обезьянка haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:44 0 0 0 0 utf-8 401 0 0 0 0 663421629 0 0 0 0 0 c 2014-03-24T07:38:25 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 17131 -1 2 wi a5b3 0 -1 0 0 52 15 0 0 272 0 -1 -1 -1 401 -1 -1 0 0 07d21f 0 [] 0 713690030010978492 14503036338486044784 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -7479382069659086597 0 Продаю бесплатно - светике Татья владельцев 1 2014-03-23T21:20:45 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://kliksa.com/iframewom http://galerid=0&high-anner.by/?search?fr=all&text=купить доме&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgo moniac.com.travellecomme exist.ru.msn.com 0 0 [292,353,7606] [5760] [18] [28] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 5 0 0 0 430 384 117 2014-03-23T05:08:11 0 0 0 0 utf-8 330 1 0 0 5177399025069094312 567587409 0 0 0 1 0 E 2014-03-23T13:28:39 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2843856621567445410 2250008690469188482 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 98 1 -7516343859483962604 1 Автомобили с проблетов онлайн на 1 2014-03-23T05:48:22 2014-03-23 22447063 3721844867 7e52689e524f80aac08d88e47e84e73d 9580 2301018798038122137 0 56 4 http://images.yandshijj-v-cheap-and http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1297 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3238011 3 2 0 0 653 693 117 2014-03-23T07:41:35 0 0 0 0 utf-8 330 0 0 0 4627722567800183582 485309851 0 0 0 0 0 E 2014-03-23T04:08:48 16 2 2 0 0 [] 3446251501 53d889ddb55859ed8adbe55c5cda7da177 45687 -1 1 wi a5b3 0 -1 17 14 261 21 0 0 1117 171 -1 -1 -1 2495 -1 -1 0 0 07d21f 0 [] 0 4313994922287858055 1100934878990743947 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7580359850907242723 0 продажа консультате с руков на AVITO.ru - страни в портименко 1 2014-03-23T19:23:45 2014-03-23 12591643 3565819545 82af9c35e16ca87ffaf7b9edfa6f42f6 239 2189462870885553765 1 97 42 https://mptri.com.tr/tr/videobox.tv/eventsnazorva/sevastok_2838##name=Новоставить http://yonja.com/etgifrm%2Ftitle,pos,p0,source,web,pre,site_slotname=Tümü&e_baski-gazetable_key=&price_valka_277565/?from=0m7g&clid=1965}/catalog%2Fsocial_hastnye-prizatsii/adme.ru/?clid=1&metrikansii/bez_opyta_sport-expresoriends.ru/ru/clck/jsredir?from=29.03.251629804b21hR1gwMGZyU013JTNEZnZTTVRBJTNEZmRTTVUUT2gtZEJhd1hHJKEMXiKRz8iFPth adultmultiki yandex.ru 0 0 [353,3121,11503,11630] [9,377,480,3676,15216,14241] [18] [28] 968 1640 57 0 0 0 0 7 73d7 1 1 1 1 LG Optimus 2853155 3 2 don шнуровногорий шёлк щёлк 0 0 592 547 296 2014-03-24T07:46:25 0 0 0 0 utf-8 401 0 0 0 8608824822004667778 816585860 0 0 0 0 0 E 2014-03-23T11:06:01 22 2 2 106 31 [397,11,9,68,14,1019,1465,1,3993,252,2] 3702300762 a18b5c7211334fac99217724417d4550d8 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 17339020080848845144 8629326841291084175 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7599913264897309014 1 Отступным низкой области в Киеве. Решение пенсиональный фото доставни.ру — Яндекс.Музыке. Мне на AVITO.ru - Продаж - визажиганы - Мы от Good, B-russianFood.com: новости на новости на 20 формация 1000005 года, долла бывший чай - НОВОСТИ - 20 купить бесплатный с персонские первый заявки деловек птичка 1 2014-03-23T12:22:58 2014-03-23 1785 1353429726 e2c0e129718aad4e93f93b5c5c066ed675 3 6510982234989222954 1 105 7 http://9111147834-video-domasma-hd.net/best-1182761198 http://yandsearch;web;;%2Fwwwww.silverdi-kading/sex/page=310&text=фильмы онлайн штор&client_id=599277a696313/album/17-yil-hake.ru/?/ coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 3701 1667 29 8 0 0. 0 0 5 qR 1 1 0 0 2765439 3 2 0 0 2793 1041 157 2014-03-23T20:25:05 0 0 0 0 utf-8 401 0 0 0 9043742639069289622 56687838 0 0 0 0 0 E 2014-03-23T22:58:27 16 1 2 0 0 [] 1764751978 72253090507ab4406a6d83717c75243e -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13306854087717566808 2241902997682127110 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7687055982568657638 1 Историентов на AVITO.ru - Поттехнику в асоціальный сайт На юго-востинга" : один бытового парфюмерили новости, дереводы. Realty.dmir.ru - Почта Mail 1 2014-03-23T16:58:38 2014-03-23 33044186 2126080452 fab776b7bdcf8dcb68cb6922724a9362 89 2034549784946942048 1 3 5 http://yandex.ru/cgi/online.com.ua/detskie-spb.blizko.ru/index.ru https://market.yandsearch/non.ru top-androeed rabota.yandex 0 0 [3,15,63,95,75,381,2182,4132,10886,16137,15261,13770] [2,15,46,123,102,507,3498,4504,15216,10157,15095,11161] [18] [28] 1846 1640 57 8 0 0.77 3 6 10 73d7 1 1 0 0 2175425 -1 0 0 0 1071 955 945 2014-03-24T11:26:33 5 1 19832 0 utf-8 401 0 0 0 5669772058100890221 888201737 0 0 0 0 0 E 2014-03-23T06:10:57 39 1 3 8971 0 [95,6,408,103,10,3,44,42,76,47,3993,209,51] 1341355226 fdd940e1982462885d4c8fb848816939 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15349291019717589336 8037358779388095153 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 0 0 -7780035710352963112 1 Голая Нагородской областической 1 2014-03-23T05:23:08 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/load.ru/#hs http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T05:29:56 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 546201586 0 0 0 0 0 5 2014-03-23T03:47:32 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 3 wi a5b3 0 -1 0 0 277 84 0 0 858 85 -1 -1 -1 1454 994 -1 0 0 07d21f 0 [] 0 4313994922287858055 18236259282679687280 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 10 0 -7790500540215037749 1 Тимошенковске на F1news: Новгородажа легководителей и ее европы - Странспорт 1 2014-03-23T10:51:56 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.139517&quickpantinoyleri http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 943 296 2014-03-23T22:22:46 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 515139757 0 0 0 0 0 E 2014-03-23T20:22:37 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 25 116 83 126 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 7982945165681148628 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -7794706186709683946 1 В пробегом - катеристочник 1 2014-03-23T16:32:20 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:59:55 0 0 0 0 utf-8 401 0 0 0 0 667659827 0 0 0 0 0 E 2014-03-24T08:03:47 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 49 20 137 36 0 0 1861 0 -1 -1 -1 1987 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 236 0 -7829278250573826232 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:56:49 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:02 0 0 0 0 utf-8 401 0 0 0 0 708852166 0 0 0 0 0 E 2014-03-24T07:37:21 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 22279 -1 1 wi a5b3 0 -1 0 0 1 28 0 0 559 0 -1 -1 -1 1476 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -7893175662663208254 0 1 2014-03-23T21:20:38 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carfiles/2014/9119&redirektor_view/27409,221&msid docviewer.yandex 0 0 [] [443,333,5760] [18] [] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 0 0 0 1 430 384 117 2014-03-23T05:08:04 0 0 0 0 utf-8 330 0 0 1 5177399025069094312 567587409 0 0 0 1 0 E 2014-03-23T13:28:33 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 314 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4149647501617670018 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 116 1 -7939467002160261687 0 1 2014-03-23T21:12:27 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogo_baskalabs.html tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:43:02 0 0 0 0 utf-8 330 0 0 1 0 529683494 0 0 0 1 0 E 2014-03-23T11:12:54 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 1 nD Tp 0 255 -1 -1 -1 -1 -1 -1 -1 -1 2253 3940 48 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 2665691200259067948 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 993 1 -7940922169703095731 0 Знакомства в Республике Дагестанцам 1 2014-03-23T11:36:25 2014-03-23 1785 994141745 9da519855c6298ca2b15a5da579246a283 207 1836964949227567248 1 42 71 http://9111147851/?&p=1&text=укладоватки закон http://yonja.com/kamuajanssories.ru/ulya-volt.ru/kino-haber.com/peugeot/230000&sp=-1&o=likest.tanka.pro/calendi-shpartaming%2Fsimferotiv-vosti.ua/oldcars coccoc.com yandex.ru 0 0 [3,5,6,15,57,91,76,175,353,403,349,3121,11503,11630] [119,3418,2455,3036] [18] [155] 1009 1367 57 0 0 0 0 0 73d7 1 1 1 92 iPad 7 1443169 3 2 психологда тура кирпич 0 0 674 1115 117 2014-03-23T19:40:39 0 0 0 0 utf-8 401 0 0 0 0 732369666 0 0 0 0 0 E 2014-03-23T02:32:34 55 2 2 0 0 [9,11,42,3,1] 990921201 f7aaf7453dcba45c62882493cd80e6fe81 -1 -1 -1 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 14359677112441125208 16829396780134885488 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7960500408286544976 0 Дикие стал автомобильники | каталог холстук, баз для мультуры по которы, услуг Россию :: Ходческие, фильм "Холодная рабочий купить 1 2014-03-23T21:20:31 2014-03-23 22422732 2271852515 56b72cde5096c0f237452cd9f670d92c 196 8579128732000753997 1 223 42 http://video/torre.com/odnoi-stimeout=1440&is_vative=0&s http://fast-topic/282039.html%3Fid%2F1000/query.ykt.ru/yandex.ru/fblogs.yandex.ru;yandsearch ficbook.net mail.yandex.ru 0 0 [5,4,372,9487] [15] [] [28] 968 1640 57 0 0 0 0 4 73d7 1 1 1 1 LG Optimus 1769215 1 0 0 0 592 838 157 2014-03-23T06:41:05 0 0 0 0 utf-8 401 0 0 0 4898301856994370214 460298661 0 0 0 0 0 E 2014-03-23T13:28:03 16 1 3 3918 6 [64,7,6,109,14,5,285,84,81,80,4] 2295410265 b140e0077981e3689a5f6973035a3b7e -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10671434186064935256 1759711880979997785 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7972070184216637013 0 Honda Bellas - Jerry - моя стулья | Новини 1 2014-03-23T21:12:45 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/bagajd-world/567765647 http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:43:26 0 0 0 0 utf-8 330 0 0 0 0 274009037 0 0 0 0 0 5 2014-03-23T11:13:16 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 132 174 11 0 462 6 -1 -1 -1 543 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 2948755971896419986 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 656 0 -8035654727346356734 1 В пробегом - катеристочник 1 2014-03-23T16:27:45 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T06:02:54 0 0 0 0 utf-8 401 0 0 0 0 426221199 0 0 0 0 0 E 2014-03-24T08:04:20 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 46 119 0 0 740 0 -1 -1 -1 1412 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -8075310065356086365 0 Hotel.jpg» на продаю карте | | Для домов.НЕТ - поиск, познавала «Весен 1 2014-03-23T21:15:10 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/ball/anasayfa.irr.ru/page=0&availaut/to/casino http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:45:57 0 0 0 0 utf-8 330 0 0 0 0 11121749 0 0 0 0 0 5 2014-03-23T11:15:56 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 927 314 11 0 282 13 -1 -1 -1 1159 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 15729620011142801520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 658 0 -8094433118608774279 0 Светские главные, долла: упражные сериалы доллар - Страница 10 сон. Женскую система Rezervative Burcundai Sanatik 1 2014-03-23T16:33:25 2014-03-23 18645263 1790084893 60057a4e582b5faf88f2c49235bc138a 8363 803723286935347333 0 42 71 http://blog/resim-6/ http://yonja.com/quests forum.shestvennik.ru yandex.ru 0 0 [353] [] [] [28] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3031088 1 0 0 0 592 1376 117 2014-03-23T07:39:47 0 0 0 0 utf-8 401 0 0 0 5718608483174516921 922710799 0 0 0 0 0 E 2014-03-24T01:25:47 16 2 2 0 0 [] 1983574728 e3ebee64fe91325ffa29ba8579b9d3ae 62835 -1 1 wi I7 0 -1 0 0 0 7617 0 0 5486 984 -1 -1 -1 15108 -1 -1 0 0 07d21f 0 [] 0 2351151346570785378 7780118309660496456 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8149355527847772674 1 В пробегом - катеристочник 1 2014-03-23T16:24:43 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:54:01 0 0 0 0 utf-8 401 0 0 0 0 778610602 0 0 0 0 0 E 2014-03-24T07:58:46 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 16 22 61 62 0 0 785 0 -1 -1 -1 1064 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 173 0 -8155806667442330296 0 Hotellot - Рыбалка.ру - все серия 1. Писточный или Турция, Гомельной банка 1 2014-03-23T21:14:08 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/ball/image&uinfo=ww-135136361 http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:44:44 0 0 0 0 utf-8 330 0 0 0 0 44551634 0 0 0 0 0 5 2014-03-23T11:14:53 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 184 499 18 0 779 7 -1 -1 -1 1948 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 5463880999052126868 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 823 0 -8156909267415167339 1 Брюки с TurboBit.net - фильмы онлайн на 2014 - Финансий 1 2014-03-23T16:23:08 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://hurpass.com/ogrevolyni http://foto-395077193b7240786&lr=11110436606 photostrana cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [9,121] [] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:47:01 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 845095662 0 0 0 1 0 E 2014-03-23T23:47:08 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 51 2009 1293 2 702 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 18315239292404426864 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 1 -8157755090657987777 1 вк — Яндекс.Почта/АndroidMag 1 2014-03-23T09:50:56 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.1401/sovmestore http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 637 296 2014-03-23T21:17:04 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 834077893 0 0 0 0 0 E 2014-03-23T19:47:34 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 11 77 91 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 9450665378893719664 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -8165890628220817396 0 1 2014-03-23T21:13 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/bagaji-demoi-fena tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:43:36 0 0 0 0 utf-8 330 0 0 1 0 274009037 0 0 0 1 0 E 2014-03-23T11:13:27 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 5 nD Tp 0 204 -1 -1 -1 -1 -1 -1 -1 -1 2055 3158 320 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 855065554003365461 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 766 1 -8207621839602888071 0 21:24:02) « Политик 1 2014-03-23T18:06:44 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/for/zilcc.ru/tsotsbank yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 0 0 0 0 430 234 117 2014-03-23T19:08:37 0 0 0 0 utf-8 401 0 0 0 0 131924827 0 0 0 0 0 E 2014-03-23T12:13:53 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 3912 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 6785386778629335136 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -8262700426099359718 1 1 2014-03-23T16:21:13 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://video/embed/68411756114414293 http://foto-395077193b7240786&lr=11110436606 amksports.ru.livemaster cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [45,333] [18,348] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:44:38 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 42215400 0 0 0 1 0 E 2014-03-23T23:45 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 49 5647 4852 2 2891 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 2532179236054953957 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8321182583543853388 0 1 2014-03-23T20:34:29 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 tranamaschools 0 0 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 0 0 0 1 1 1 117 2014-03-23T06:05:11 0 0 0 0 utf-8 401 0 0 1 0 122853544 0 0 0 1 0 E 2014-03-24T14:17:35 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 2998 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8456596711470618355 0 Samsung Galaxy tarisme online Search resmi Sistem TurboBit.net.ua - DX 130 000 рубля игры 1 2014-03-23T06:47:03 2014-03-23 1785 901953317 a4a276d0ea345c74148750a6919a36dad5 207 1836964949227567248 1 42 71 http://rutube.com.tr%2Fgazpromo38.ru/search?lr=2&rpt=simage avito.russia.rust.net.ru 0 0 [] [3,160,864,2455] [] [] 1009 795 57 0 0 0 0 0 73d7 1 1 1 92 iPad 7 1443169 0 0 0 0 674 927 117 2014-03-23T14:25:27 0 0 0 0 utf-8 401 0 0 0 0 833653080 0 0 0 0 0 E 2014-03-23T19:03:13 55 2 2 0 0 [9,11,42,3,1] 1061339287 20bfc7e7245325fdac7e99f96f7dcbe6 -1 -1 -1 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 13975993063278569584 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -8472838923458193599 0 Личный компаний: "В бороны панели 1 2014-03-23T10:26:41 2014-03-23 1946 2822174503 83f4c4017c625c30615e808b9b553cd25a 950 1989946518134869356 1 42 71 http://avito.ru/shoes-with-avciliusersView yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 1 0 0 0 0 592 1376 352 2014-03-23T04:42:49 0 0 0 0 utf-8 401 0 0 0 0 285947293 0 0 0 0 0 E 2014-03-23T02:39:02 16 2 2 66 0 [2193,82,265,14,95,6,100,9,72,3275,1,70,11] 3219490004 5c6e7add6158bbed0699cbe973042ea2ef -1 -1 -1 nD Tp 0 -1 0 0 0 2817 2921 2527 30475 0 -1 -1 -1 29211 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 15977284023899318384 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -8555937822471823535 0 1 2014-03-23T16:34:48 2014-03-23 18645263 1790084893 60057a4e582b5faf88f2c49235bc138a 8363 803723286935347333 0 42 71 http://blog/resim-6/ forum.shestvennik.ru 0 0 [] [] [] [] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3031088 0 0 0 1 592 1376 117 2014-03-23T07:41:33 0 0 0 0 utf-8 401 0 0 1 5718608483174516921 922710799 0 0 0 1 0 E 2014-03-24T01:27:26 16 2 2 0 0 [] 1983574728 e3ebee64fe91325ffa29ba8579b9d3ae 62835 -1 1 wi I7 0 2002 -1 -1 -1 -1 -1 -1 -1 -1 25844 25608 85 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7780118309660496456 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8563276039202285702 1 Голос видео 1 2014-03-23T05:23:15 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/triket/4746734017811fc051dd0e57 http://yandex.ru;yandex.ru/?area=ShowForum.ru/look/event cdo.e1.ru.plays олины.рф 0 0 [] [] [] [] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 -1 0 0 0 1460 549 574 2014-03-24T05:30:03 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 215720129 0 0 0 0 0 5 2014-03-23T03:47:38 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 5 wi a5b3 0 -1 0 0 289 1 0 0 94 20 -1 -1 -1 488 414 -1 0 0 07d21f 0 [] 0 18236259283221887320 16190469452247787632 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 9 0 -8599500047480121116 1 Брюки с TurboBit.net - фильмы онлайн на 2014 - Финансий 1 2014-03-23T16:22:36 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://hurpass.com/ogrevolyni http://foto-395077193b7240786&lr=11110436606 photostrana cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [9,121] [] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:46:24 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 42215400 0 0 0 1 0 E 2014-03-23T23:46:39 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 18315239292404426864 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 1 -8602651835226449867 1 7 самая мамки для ваше может по суперма Страница 719 тыс. ответов 1 2014-03-23T12:16:45 2014-03-23 1785 2088542490 921202579dbab4e58eddb04f693854b3 57 9499572962667875395 1 56 122 http://911114788/38/celebekgaleri.hurriyet http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/downloadTo=&orderby=RelAmountry_id=22&ved=0CAUQjB0&url=http://money?rand_cruitstart=444305.952058.13951639370f280c133ad16ce2c79e7cab93c5a23X3 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 11 6ee9 1 1 0 0 2366248 3 3 схема сумерки 5-6 летовой устанния 0 0 1627 708 157 2014-03-23T22:11:02 5 1 24018 0 utf-8 401 0 0 0 8929720244896745512 460839648 0 0 0 0 0 E 2014-03-23T22:21:38 0 0 0 0 0 [] 114378192 d7364e6a7ddcbcf6dcccca7bd6b2807a -1 -1 -1 nD a5b3 0 -1 0 0 626 189 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6784513853171268256 7631625543297538199 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8607066140185696713 1 Отслеживаны - №1065 - ekşi 1 2014-03-23T16:28:07 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/weatherson.html#78efe9869.shtml?wmid=143&srcid=UxB0pAAA6I9d0CWaWE%3DfrSMw http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T06:03:11 0 0 0 0 utf-8 401 0 0 0 0 813102074 0 0 0 0 0 E 2014-03-24T08:04:40 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 15 110 78 102 0 0 2060 365 -1 -1 -1 2397 1361 -1 0 0 07d21f 0 [1555356] 0 6331778101536488133 11618756116771170416 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 8 0 -8611399304547034181 0 Современа - Футбольших 1 2014-03-23T16:22:51 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1188c56ff8058343682.1_29_et._30108879a9aa61ea73752719 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vdmlyLnJ1Lw%3D%26engine_volume2=40&sid=6&ved games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:23 0 0 0 0 utf-8 401 0 0 0 0 356217277 0 0 0 0 0 E 2014-03-24T07:51:41 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11767694963832011096 2937255180427836822 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 111 0 -8664317093542350977 1 Афиша - купить готовой менение сиUçuşlardo, Yazlık Burçlar - astopol.info - Classion AAAQ (250Hp) CVT (140 тыс. км., смотр объявлению в автобусов - Екатегория России, клиент 1 2014-03-23T20:34:25 2014-03-23 32821467 1157253311 482b48fd303d64bba86ec7965c6ed6dd67 207 2460387875241374418 0 56 4 http://rabota.ru/show%2F2014 http://read.php?act=8&ved=0CC85ViwdY8L4WG9pLWlKS0RJU3AeAxJuh wwww2.sports m.auto.yandex 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3815009 -1 0 0 0 1136 593 117 2014-03-23T06:05:10 6 0 39566 0 utf-8 330 0 0 0 7635035874316412186 1000586617 0 0 0 0 0 5 2014-03-24T14:17:28 39 1 2 70 51 [10,42,1,22,3,112,3773,103,5,51,15,4,3212,47] 1549383847 ed3b609e3397b12c56cca4469c881c38 17234 -1 7 wr a5b3 0 -1 0 0 179 57 0 0 212 103 -1 -1 -1 613 -1 -1 0 0 07d21f 0 [] 0 3832538266798636143 15412096237897967728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 22 0 -8665395302091187491 0 Современа - Футбольших 1 2014-03-23T16:22:43 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=106/cid=191&rpt=simages%2Fthumb%2Fimage&lr=157 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20v&com_id=0&body_type=0&clid=1980&s_tg=&tariniz games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:12 0 0 0 0 utf-8 401 0 0 0 0 588577730 0 0 0 0 0 E 2014-03-24T07:51:28 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 1560098688466543352 11767694963289811056 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 121 0 -8673158852214232182 1 Голая Нагородской областической 1 2014-03-23T03:51:08 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/load.ru/#hs http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T03:26:28 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 270908049 0 0 0 0 0 E 2014-03-23T02:24:56 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 55446 -1 3 wi a5b3 0 -1 31 93 484 10 0 0 546 176 -1 -1 -1 1135 -1 -1 0 0 07d21f 0 [] 0 4313994922287858055 18236259282679687280 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -8725981425210147796 0 Аудио ЭХО Москве, ножений вопросова нашлось 397 1 2014-03-23T02:41:19 2014-03-23 15665476 1648690379 04eb9547b256147702f34bd2464beaf0 196 1425686864647023563 1 56 3 http://ivona.ru/top.rbc.ru/moscow=&q= Erection27=&toName=false&morpholog/vakansition&ref=city/daily http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/news.rambler.ru/search/ex09KBBcpWRDlaYTZYNEJHZDlOb3VHdjNWRXg5UFBpV2pFQ1qUQ&b64e=2&output=uds_app_id=19855479&view?ad=44&s_yers.don.ru/context=Sex&geo=0&s_tg=&offset=2.\\tПоповорого план кубе&clid=20639&keyno=0&l10n=ru&mc=1.58496250001978934&Lt=6&refresh=1&has_photofunia.ru/#!/video/search;web;;%2Fweb%2Fitem%2Ftitle,pos,p1,source=web&text=текстра фото 2014/view/125329d08a194e758644-500h_bWOg&bvm=bv.62922401,d.bGE&cad=rjt market.ru.platnye-igrydlja tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [] [] [28,267] 1622 1583 29 8 0 0. 0 0 10 73d7 1 1 0 0 1808122 3 3 интернет магазин финес и физическу о приколеснока, валерия 0 0 2011 768 157 2014-03-23T04:34:29 5 1 19832 0 utf-8 330 0 0 0 6243379005186084238 8777098 0 0 0 0 0 E 2014-03-23T21:50:32 16 2 3 854 0 [2193,82,265,3275,347,14,72,925,100,3,80] 1245455042 891bf70623c57bfdd5e9d09d616390eb -1 -1 -1 nD Tp 0 -1 0 72 206 84 12371 0 -1 0 -1 -1 -1 18 -1 -1 0 0 07d21f 0 [] 0 2560641399298933162 12328847313983957104 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8746280778383767298 1 Поздравоохранул 1 2014-03-23T13:55:17 2014-03-23 1785 3609570631 e9fd65b19c652c4ce85594f2fcd0c7db 196 593475812248875581 1 227 105 http://911114783342507357617c9eb279bb4f2642 http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/video/index.ua/pole-2167ba0f7c355305-nam-v-pid2-290622263757&m=3,5&Volume[1]=&extras[24]=0&eae=0&clid=22876.26.2.0.html&docid=onair.com/galeride/Kariy_el/zafiruet-onlajn_tr_5Gn0cFj_bANt_S8ROdvfrOg6pCgU5XY2P3MtHkZQNDILDhspeAAhWjJJgkFq5HXGrEC5GmjXQ5bGV3TTVeklJakNwelhKc0hDcjFTVTFWb180ff0d9&url=http:%2F%2Foktavito1296_3346d0f0;dcid=ya_wireleva.ru/yandex.ru/?action=page-67600&groups[]=17704&groups[]=13&msid=91cc2424241d2cd8128&lr=97541¬custom_key=&haggle_AO_Mondelevizit-dvigator.tv/tnt/?region=cadf2922401,d.bGE&cad=rjt coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 523 617 57 0 0 0 0 44 73d7 1 1 1 0 3684816 3 3 алекс старт 20кг купить десан автобазаречь создать 0 0 592 310 157 2014-03-23T22:38:38 0 0 0 0 utf-8 401 0 0 0 8382449486543033592 279598542 0 0 0 0 0 E 2014-03-23T03:43:12 16 1 3 119 4 [76,1] 3886548777 f69565e66e18aeaf2059a8b6495c723613 -1 -1 -1 nD a5b3 0 -1 0 28 22 9 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 4244945004353045279 5931335146677711795 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8931522039698213139 1 В пробегом - катеристочник 1 2014-03-23T16:28:08 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:56:58 0 0 0 0 utf-8 401 0 0 0 0 126947969 0 0 0 0 0 E 2014-03-24T08:00:54 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 22 23 62 61 0 0 996 0 -1 -1 -1 1043 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 240 0 -8933906015862790098 1 Афиша - купить готовой менение сиUçuşlardo, Yazlık Burçlar - astopol.info - Classion AAAQ (250Hp) CVT (140 Hp) 2006 купить компании Украина вытащил о Москвы :: Голосуток в 1 2014-03-23T20:34:36 2014-03-23 32821467 1157253311 482b48fd303d64bba86ec7965c6ed6dd67 207 2460387875241374418 0 56 4 http://rabota.ru/new.mcko http://read.ru/yandsearch wwww2.sports m.auto.yandex 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3815009 -1 0 0 0 1136 593 117 2014-03-23T06:05:24 6 0 39566 0 utf-8 330 0 0 0 7635035874316412186 28170112 0 0 0 0 0 5 2014-03-24T14:17:36 39 1 2 70 51 [10,42,1,22,3,112,3773,103,5,51,15,4,3212,47] 1549383847 ed3b609e3397b12c56cca4469c881c38 17234 -1 7 wr a5b3 0 -1 0 0 106 58 0 0 92 34 -1 -1 -1 148 -1 -1 0 0 07d21f 0 [] 0 15412096238440167768 13753603333183694960 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 19 0 -8939870126547196026 1 Комная прессан Партира, акциях — фанфики к экзаментально и части Донецка | РБК daily Digital acro 1 2014-03-23T16:02:25 2014-03-23 1785 2922543887 1c01b17e6ee3c1a01828ecd318a6b581 59 8723854704339558313 1 56 2 http://9111147842,0.01178.1394999-prepovedeki http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net%2F1b.jpg","photos&p=3&page=18413930_handguides[]=2&sign=cf8f01c12489e0a7&uuinfo/view/93190876323017&l=sem&sign=213&nore[2]=0&search?p=законы финальных треуголоваров владимирова туалеты 10 дневники андроид&lr=20525944&lr=1637&keyno=0&room=0&page=576&m[]=100&search/retsiatoust/avtobank.ru/#!/kemeye-karechenkoi-denegro/brando.ua%2F17%2F204a%2F&ei=0-ochekiev.ua/opinion%26wheelpUTdBQUhtbTZhSzItYVZOOUjhfNF9vZDdGVzLzAwMzA3L3BwLzQxNjY2ZHpnY2VWxiCHsuJClHPnYMQVoxbSVTUFJTElsakI4WFp6NHNHJRDRhOU1LR3BuSmJYYj9tOUmxUOXMtVlN1TjNtVGc1c032a1b3672 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1297 1010 57 8 0 0. 0 0 18 73d7 1 1 0 0 2992777 3 3 детский номер электрообная игры скорота фото кало 0 0 1366 391 157 2014-03-23T00:56:51 0 0 0 0 utf-8 330 0 0 0 4847654860042290847 969371126 0 0 0 0 0 E 2014-03-23T02:42:26 55 1 3 12257 0 [10,9,112,68,365,76,260,22,1,2817,3,3540] 2380200681 0735dfb043075c30693475487bf24a49b7 -1 -1 -1 nD a5b3 0 -1 1 55 84 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 5849787649459211608 17662636599668303984 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -8964700706069591280 1 Салата 10. Цензор видео, онлайн (все забилей с пройти в Мельного 1 2014-03-23T04:19:17 2014-03-23 22447063 1796001934 3ae86dda5556b7f4ef38aa8077f6825a 15887 1644674872295047497 0 3 4 http://images.yandex.ru/neo2/#inbox/&dates=0&run[1 http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 2 [353] [] [] [28] 1587 555 57 10 13 0.77 0 0 25 73d7 1 1 0 0 3996694 3 2 0 0 1713 555 117 2014-03-23T19:14:50 0 0 0 0 utf-8 401 0 0 0 0 939725270 0 0 0 0 0 E 2014-03-23T02:46:14 0 0 0 0 0 [] 1907295579 dc652869f8d8eff9ed5b5daa5374b163 5038 -1 3 wi a5b3 0 -1 305 0 452 30 0 0 4029 164 -1 -1 -1 5023 6719 -1 0 0 07d21f 0 [] 0 4313994922287858055 9730402382055663728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -9065033574850035452 1 Серия 9 - 2013. Дети@Mail.Ru — социалист 1 2014-03-23T16:20:14 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://yandsearch?lr=35&rpt=simage&lr=66&text=розаврилова http://google.com/games.mail.ru ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 3 2 0 0 1994 561 117 2014-03-23T06:43:38 0 0 0 0 utf-8 401 0 0 0 9007862211105217806 42215400 0 0 0 0 0 E 2014-03-23T23:44:06 16 1 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 47 40 590 134 0 0 2110 -1 -1 -1 -1 -1 3460 -1 0 0 07d21f 0 [] 0 1930741556500475038 9207967286803898539 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -9186659792231785281 1 Настройствоваться частных (б/у) автор о продукции; развития детьми - Грузов 1 2014-03-23T09:48:08 2014-03-23 1785 1341248658 6366e773993d35514d6b846f79b34292 183 626923241082458450 1 56 4 http://9111147832977565.html?cat=420801;label=perioda http://go.mail.yandsearch coccoc.com yandex.ru 0 0 [3,5,6,15,57,91,76,175,353,403,349,3121,11503,11630] [119,3418,2455,3036] [18] [155] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3500889 3 2 0 0 1997 548 117 2014-03-23T17:20:17 0 0 0 0 utf-8 330 0 0 0 8416052423457712746 215289560 0 0 0 0 0 E 2014-03-23T23:14:07 16 2 2 13621 14 [14,72,80,925,370,109,7,285,3274,101,6,66] 1160820115 524ee8575739a6149a641e6f4fbc6f7b -1 -1 -1 nD a5b3 0 -1 9 45 109 42 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8578740285396261239 4548538545130905100 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 - --- !sql13 -- -4632802378222380466 1 null Acoper «labilir mitsubishi в Липецке на Batak 1 2014-03-23T16:23:07 2014-03-23 27528801 2723656327 7fc6e55c74e3029834ae16bf67d8cb9f4f 15887 2224216775939367616 0 56 4 http://rutube.ru/patianu http://love/v012/04 footki.yandex.ru m.fotoalmeta 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3238011 -1 0 0 0 1136 555 117 2014-03-23T06:04:09 5 1 19832 0 utf-8 401 0 0 0 0 1018986580 0 0 0 0 0 5 2014-03-24T08:02:03 16 2 2 0 0 [] 2529497135 30d877a66ec8945d1e4d056e51e3de35 18372 -1 2 wi a5b3 0 -1 4333 1332 747 2472 0 0 7395 285 -1 -1 -1 16115 -1 -1 0 0 07d21f 0 [] 0 307141980878902893 338497349483004835 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 47 0 -4716566754160586468 1 Тимошенковске на F1news: Новгородажа легководителей и ее европы - Странспорт 1 2014-03-23T10:52:43 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.139517&quickpantinoyleri http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 943 296 2014-03-23T22:23:20 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 278406132 0 0 0 0 0 E 2014-03-23T20:23:36 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 0 0 1 2 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 7982945165681148628 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -4755557211339260600 1 PwC 1 2014-03-23T16:25:12 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/index?appkey=&m[]=6&frommail.yandsearch?text=никол в про и невиннисаж http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T05:54:31 0 0 0 0 utf-8 401 0 0 0 0 346099590 0 0 0 0 0 E 2014-03-24T08:01:40 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 59 78 0 0 2251 0 -1 -1 -1 3362 798 -1 0 0 07d21f 0 [1555356] 0 6331778101536488133 6157439411447496521 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -4770046483886760822 1 Mercedes-Benz e-клас. Можно купить 1 2014-03-23T16:26:50 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/weatherstyle/29/127737064&domain=hurriyet.com/volgogram/22/10/Default-tistings=535674/atturkiye http://mynet.com/mate-guzellileridetay/80508/bio/video?q=породажа хабаров с технологических mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T06:02:18 0 0 0 0 utf-8 401 0 0 0 0 782597291 0 0 0 0 0 E 2014-03-24T08:03:20 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 96 115 252 0 0 6165 306 -1 -1 -1 8173 1287 -1 0 0 07d21f 0 [1555356] 0 8744694472066974558 13107909194127435888 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -4831817364509130205 0 Сериалы 1 2014-03-23T20:34:29 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 http://yandsearch.php?id=209516817950 tranamaschools top.rusfisha.msu 0 0 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 -1 0 0 0 1 1 117 2014-03-23T06:05:12 0 0 0 0 utf-8 401 0 0 0 0 6030782 0 0 0 0 0 E 2014-03-24T14:17:35 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 -1 0 0 3018 0 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11033243704253757784 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -4835752945340096735 0 Продать, предложений — AVITO.ru. Часть бесплатно онлайн Фильма «T+2» (286): 1 2014-03-23T12:16:06 2014-03-23 1785 44808656 0442e62a5c5cb134d699fbae72bc9c588b 7 454237030118460538 1 3 3 http://91111478334250732/?page=20&i=1&parenk=&changed http://yandsearch;web;;%2Fwwwww.silverdi-kading/site_id=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJpvc9LIo8yqpc1a7i7QPYfRisPDxcOAMMjcyNjQ3MTIxfcSMTQ0Y2U4OWQ5WWVFUnFSWhVTUNjVmN2N2 coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1297 555 29 8 0 0. 0 0 17 73d7 1 1 0 0 1384917 3 2 0 0 1366 1019 209 2014-03-23T20:26:57 3 0 58982 0 utf-8 401 0 0 0 5636140508787295899 929361662 0 0 0 0 0 E 2014-03-23T22:24:51 55 2 2 675 0 [4,5,80,82,2193,285,265,76,14,10,112,3,9] 46871203 be72ce3c669bd45ee99b8e038d989526 -1 -1 -1 nD Tp 0 -1 0 0 19 11 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10605622915436919128 11933878103501891696 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -4837660920166529023 1 Сбербанк ОнЛ@йн 1 2014-03-23T13:27:36 2014-03-23 1785 280750947 99ebef8641c09581865c6e5c72b33f05895a 19973 322556318799166472 1 56 2 http://9111147842,0.01178.13950555&lr=66&img_url http://kelebekgaleri.net/besplayer&join.ua/rezerval=1&bodystyle.com/8-marter97.org/remont/innopoisk.ru/news.mail.ru/video/search.xml?categoryId=5000&year][max]=1002-fh-467-pd-1&rpt=simages.yandex/uchita.ru/view/133-ukrainy-duoi tronika.html?l=prod.dp.ua/?page/61307-fh-467-pd-1&p=17&uinfo.aspx?wt_rp=27&hid=91651068.html&lr=213&text=search?lr=2177467203159_5500000191&text=бекление&filmId=P90GJOSRzJSNWM%3DfnSZA%3DfeSfgSMQ%3DfoSfpSYmJTNFRFJzVjdkRNQ2VWRoUFNuTDJPZnJ0U3ZnhaMlJ1bEhoVTE2WFJRTzJ1bUl9HP3J5ftXDKDyocZuDO7C4iTUDb8chOvsX2R1YmU9ecE4j44QTjhNRDRpRE40ZFdElB&b64e=2&sign=89846355c1c50b406146965/36733f0427100&sfrom=592&m[]=5431ecedes_businiz-view957448172/?from=yandex.ru;yandex.ru/a-sezon coccoc.com sprashivai.ru 0 0 [5,353,3121,11503,11630] [119,3418,2455,3036] [18] [28] 1339 555 57 8 0 0. 0 0 29 73d7 1 1 0 0 1412357 3 3 ходячие раскрасны 0 0 1997 891 209 2014-03-23T20:32:22 0 0 0 0 utf-8 330 0 0 0 8168139702751979718 408832541 0 0 0 0 0 E 2014-03-23T23:57:58 16 2 2 2 0 [9,3274,1,14] 421494715 f20bb2faf420abe95559d6bcfe354d05 -1 -1 -1 nD a5b3 0 -1 0 35 58 56 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 18126425332249278808 14950347525130479728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -4850926408483995792 0 Современа - Футбольших 1 2014-03-23T16:23:19 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1_844_128338&text=ниссат б3 2.0 скачать http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vcG5ld3MtUUov2KB1GZjamJIeW1nQ3NGMkVMNEZBa3hpZ2VzLzI3L25ld3MvMjgw games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:46:01 0 0 0 0 utf-8 401 0 0 0 0 583663939 0 0 0 0 0 E 2014-03-24T07:52:16 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11875920750199978328 4452926215554207674 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 107 0 -4926263996196211822 1 В пробегом - катеристочник 1 2014-03-23T16:21:04 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T05:53:35 0 0 0 0 utf-8 401 0 0 0 0 1056965251 0 0 0 0 0 E 2014-03-24T07:57:58 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 21 49 40 104 0 526 0 -1 -1 -1 1986 1645 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -5017198962525199154 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:57:36 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:36 0 0 0 0 utf-8 401 0 0 0 0 658682892 0 0 0 0 0 c 2014-03-24T07:38:20 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 17131 -1 1 wi a5b3 0 -1 0 0 1 3 0 0 241 0 -1 -1 -1 410 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -5105151702377877258 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:12:37 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:43:14 0 0 0 0 utf-8 330 0 0 0 0 419812360 0 0 0 0 0 5 2014-03-23T11:13:03 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 2 nD Tp 0 -1 0 0 287 327 12 0 529 1480 -1 -1 -1 1501 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 750 0 -5138975406131988633 1 Сбербанк ОнЛ@йн 1 2014-03-23T13:28:24 2014-03-23 1785 280750947 99ebef8641c09581865c6e5c72b33f05895a 19973 322556318799166472 1 56 2 http://9111147842,0.01178.13950555&lr=66&img_url http://kelebekgaleri.net/besplayer&join.ua/rezerval=1&bodystyle.com/8-marter97.org/remont/innopoisk.ru/news.mail.ru/video/search.xml?categoryId=5000&year][max]=1002-fh-467-pd-1&rpt=simages.yandex/uchita.ru/view/133-ukrainy-duoi tronika.html?l=prod.dp.ua/?page/61307-fh-467-pd-1&p=17&uinfo.aspx?wt_rp=27&hid=91651068.html&lr=213&text=search?lr=2177467203159_5500000191&text=бекление&filmId=P90GJOSRzJSNWM%3DfnSZA%3DfeSfgSMQ%3DfoSfpSYmJTNFRFJzVjdkRNQ2VWRoUFNuTDJPZnJ0U3ZnhaMlJ1bEhoVTE2WFJRTzJ1bUl9HP3J5ftXDKDyocZuDO7C4iTUDb8chOvsX2R1YmU9ecE4j44QTjhNRDRpRE40ZFdElB&b64e=2&sign=89846355c1c50b406146965/36733f0427100&sfrom=592&m[]=5431ecedes_businiz-view957498&answer=&channe_2010_hand.ru/chase/aleksey510376 coccoc.com sprashivai.ru 0 0 [5,353,3121,11503,11630] [119,3418,2455,3036] [18] [28] 1339 555 57 8 0 0. 0 0 29 73d7 1 1 0 0 1412357 3 3 ходячие раскрасны 0 0 1997 891 209 2014-03-23T20:33:18 0 0 0 0 utf-8 330 0 0 0 8168139702751979718 668154955 0 0 0 0 0 E 2014-03-23T23:58:33 16 1 2 2 0 [9,3274,1,14] 421494715 f20bb2faf420abe95559d6bcfe354d05 -1 -1 -1 nD a5b3 0 -1 0 0 14 2 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10100085969557141848 14950347525130479728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5291067970604774260 1 Максай игру роботы, купить в интернет-магазин спецпредства воорешебниках - вакансии лет — Сеть легководставито 1 2014-03-23T05:07:16 2014-03-23 15665476 1087886866 269c5c27e4946a667f78888c9f75b35c6e6e 49701 334328984627246488 1 56 5 http://mail=1&search?text=гугл перевод старый сайт с шоп&strirodam_spanie-podkRPaXptc3hrVndyR2JaRFUxWTJ4aFVG http://hood_search&event_id%3D84840.2402105-50&uuid=&status=0;orderevery-club.me/berkusurulu-bolum/2522&language=2014&page2/?go market.yandex.ru.msn.com pegasha-pogoda 0 0 [353,3121,11503,11630] [] [] [28] 1174 809 29 8 0 0.77 0 0 15 73d7 1 1 0 0 3560775 3 4 погода на рассаж 7 0 0 1962 676 157 2014-03-23T06:41:45 0 0 0 0 utf-8 401 0 0 0 0 419939021 0 0 0 0 0 E 2014-03-23T00:07:02 55 2 3 32 103 [2193,265,82,6,95,288,14,100,3275,72,3,76] 1477819126 1be50e4c7673a88c47cace3e47478733 -1 -1 -1 nD Tp 0 -1 0 0 3 30 61 0 25 -1 -1 -1 -1 -1 52 -1 0 0 07d21f 0 [] 0 8851578361410429238 10331158898916649072 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5305634367927610522 0 Современа - Футбольших 1 2014-03-23T16:23:48 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=19554212079d1b7cbb8a6a2.jpeg%3DfeSfgSYmI%3DfdSNTU2feSMzAw http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb21f3fc3aaa24dc63a21xZGVqSE1GcFQxUkJN&b64e=2 games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:46:25 0 0 0 0 utf-8 401 0 0 0 0 167969307 0 0 0 0 0 E 2014-03-24T07:52:45 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 3480781366221101891 1661165035492520768 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 127 0 -5309644206721379498 1 Письмоградской 1 2014-03-23T20:55:44 2014-03-23 27941559 1767624985 a7da21818799159fb389b18d4373596b 225 2721568806677236614 1 56 3 http://patia_spark ns7&search?clid=9582 http://hood_search&event=list/casing-cap tvmir.nu pegasha-pogoda 0 0 [353,3121,11503,11630] [5,7,73,400] [18] [28] 1174 809 29 8 0 0. 0 0 17 73d7 1 1 0 0 2179614 3 4 герб марта спицам терми 0 0 1731 391 117 2014-03-23T05:14:07 0 0 0 0 utf-8 401 0 0 0 0 777203605 0 0 0 0 0 E 2014-03-23T13:07:48 0 0 0 0 0 [] 1852761877 b016f1c7c4de86f1c384f8d63570a742 -1 -1 -1 nD Tp 0 -1 0 0 460 199 31 0 1530 0 -1 -1 -1 2846 -1 -1 0 0 07d21f 0 [] 0 15279646984699132248 8418678203875343898 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5420983836600446463 1 Продажа Toyota / Результиворк! 1 2014-03-23T03:35:14 2014-03-23 1785 1950378921 c2602efb1311636cf875df404f3d6529 51 3206084085559802010 1 56 4 http://9111147854&change_key=506d9e3dfbd268e6b6630e58&domain=hurriyet http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ukrayila-yenisafak.com%2Ftitle,pos,p0,source,web&text=кредиторожный краска длинны онлайн&uuid=0&count=135238478226d6ff8ed00dMU7UvmIoQmkzfPmqekNCSExDRVoxZEcxcGJsN3kxUUxhalleonary_to=2014 sony_price_usd[1]=&proigry-dlya-suka.ru/sankt-peterburgskaya_oblast_volume[2]=&color_id=0&year[1]=&private/en/sessuary?p=90&q=1395592f72162c019715ade9befae9562669,"first_name":"Альберт&clid=195545452&lr=109aa7f2a8bc3d75c95724332 coccoc.com tanks.co.il 0 0 [] [119,3418,2455,3036] [18] [] 1297 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 906571 3 3 торта самарекс по оптом из неполных года выхода скалы фотошоп 0 0 1366 593 262 2014-03-23T16:44:09 5 1 14470 0 utf-8 401 0 0 0 4808217662922694360 7065527 0 0 0 0 0 E 2014-03-23T16:14:51 0 0 0 0 0 [] 2137570165 46a233a0e33f11c4fee726c7303f44d5 -1 -1 -1 nD a5b3 0 -1 1 173 292 430 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8411534898364058195 7194548681272151755 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5520081962326584152 0 Ремком крае национны [AVC], VA, Любой в Москве крымская историгина домашними рукодекс: нашлось 3 млн ответов 1 2014-03-23T09:45:17 2014-03-23 1785 1242723321 2684553455bcb892472422768c7b4b2f 13437 3074483810024357617 1 56 2 http://9111147842,0.01178.13956069c61cc8Apdghvwm6ThW9Y4QOdYgU_GUIzigLM8W6g0QMWtNUXI&where=all&filter coccoc.com 0 0 [] [119,3418,2455,3036] [18] [] 1622 1583 57 10 6 0.70 0 0 14 RT 1 1 0 0 3039699 0 0 0 0 2011 726 296 2014-03-23T18:39:54 5 1 19832 0 utf-8 330 0 0 0 8207959345338106813 992900079 0 0 0 0 0 E 2014-03-23T20:03:28 22 0 3 13732 6 [6,219,1549,7,3003,501,999,18,1,36,25] 2143328713 fcb87cf6e44fbb0e1d8456015c663d3786 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4127982339363240333 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 96 0 -5559361293957300414 0 Женская библи (сериал "Фиксика 1 2014-03-23T18:17:33 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/forums.kuban.ru/volgogradmir http://yandex.net/legkovye-avtorii/118534731400928 yandex.ru tury.ru.msn 0 0 [85,350,1212,3849,8358] [106,323,1092,3676,13188] [18] [28] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 -1 0 0 0 592 503 117 2014-03-23T19:19:06 0 0 0 0 utf-8 401 0 0 0 0 647923622 0 0 0 0 0 E 2014-03-23T12:26:14 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 1166 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 884992008826344665 4724303053209794720 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -5619100536162139501 0 QIP.RU - Гайд Dota Construment Steel, Красности стильни госдеп США Джинс | Вакансии. Одежда деловек пау газета.Ru — смотреть лительность ерально и в Новости Украина, фото - View Film izle Çizmet Kay - VAZ, Gereketim Dinlementique – сайте AUTO.RIA - Фишер Успевай 1 2014-03-23T12:34:12 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://9111147854&change_key=506d9e3dfbd268e6b6630e58&domain http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/view_intext=как в москва режим умом заводоровать онлайн бесплатно в любимому языку 4 класс порно встрашный сайт длинновая папины мамедостовск-орловского ли реимущественны и холодилер шеференков&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XsltfjMh8My-4Mp2Kzonmlr9Oj-tkmzK1-_9gEp7c156bcf985&text=sony?p=11&cbv=r2014/03/19/bigparams/174780954.24549/2679&page10/?&p=36&uinfo/index.ru;yandex.ru/krasnodar.org&from=yandex.ru/viewly&kinda-raduga/arams.com/haber/10/155-02-20053.4678203611557414e2725-derby_day=2014&clid=11985&lan=ru&mc=4.804252639b7ba5981421beb&ui=uv3lovAaf3sLhCa43t3Qfk-sdelaya/sobaki/?page/dl/секс поручную под люди&lr=213&text=гарнир__Абсолютное подтяжки&lr=108444071-fiksiki-6-chere=all&filter_id=13451&lr=8&text=диета авито блей&lr=66&bpp=7&i=1&when=2009%26nord%3D%26extras[14]=0&extras[27]=29066462.0819084f155715641255491/dating/used/sale%26d%3D26%238242_259066068085843223.html?ia=M&lf=M&af=18&pw=1&target=search.phtml?pid=89§ion1[3]=&searchastnik.ru/my/hit_list/work.ua/clck/jsredirected=13859&text-albinoy-kutladikavkaz/makler.ru/aclk?sa=l&ai=C9QzwyeUQVCiFZB79rQYLQ1GPgN7Qi82fGpeBl0LLWFkQlRDAlQjUlRDAlQjklMjQ5dWFWOW9iM1I1WVhKdld5NDV1MOXXox_OxcFDY-uop-thersoy-tonkiimir_snovachale-secim.html#/battlemena.avito.ru/cars.auto.drom.ua/manage=eyAidW5yZWFsdHkvMjAxNDAzMTgiO3M6MjoiaWQiO3M6ODoiMjAxNzE0NDk0NzUva$84NzI2Mjk0NzttYAw&usg=AFQjCNGheBpruD0hte0FyaUqCNDk3DqQvygvalcenkin/offers.xml?category_id=199555453&banner_pf[HM_S106Wpb7rjrgxoAGGkuc2svdGhlYXRyemst=my.mos.ru/yandex.ru/companoritecomm coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 орхидея в мирандашом ценарисы гузеевой мото рисунок и простройщика фото с поздравлений телеканка фото 2014 год собласти ужгород 0 0 1389 884 157 2014-03-23T22:34:54 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 846684171 0 0 0 0 0 E 2014-03-23T22:48:13 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10287839608055806296 17253681420892301424 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 89 0 -5711297485370126517 1 Голос видео 1 2014-03-23T03:51:10 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/triket/4746734017811fc051dd0e57 http://yandex.ru;yandex.ru/?area=ShowForum.ru/look/event cdo.e1.ru.plays олины.рф 0 0 [] [] [] [] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 -1 0 0 0 1460 549 574 2014-03-24T03:26:32 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 585440942 0 0 0 0 0 5 2014-03-23T02:24:58 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 55446 -1 5 wi a5b3 0 -1 0 0 454 3 0 0 78 37 -1 -1 -1 386 486 -1 0 0 07d21f 0 [] 0 18236259283221887320 16190469452247787632 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -5733212747584636203 1 Как лучшие руках, часа в Красноярска обновлетворящих » Купить качесть по г. Москве на AVITO.ru (Недвижимость на AVITO.ru. Стулья салоны в Владимир 1 2014-03-23T13:20:49 2014-03-23 1785 2501928799 4a31e16945f0d802d9dc0705979c4672 239 4517116963127554624 1 79 4 http://9111147811.html?v=1393_0_251311310 http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 2081850 3 2 0 0 1997 593 441 2014-03-23T03:13:35 0 0 0 0 utf-8 330 0 0 0 7146369058166890546 793191128 0 0 0 0 0 E 2014-03-23T23:56:34 16 1 3 12333 0 [14,1,6,501,119,72,9,510,3540,925,22,11] 2976974125 a3831324980206d1415afea636cc7635 -1 -1 -1 nD a5b3 0 -1 92 125 121 201 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 12557946248999135344 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5772344969531870975 1 «Сбербайджане, вакансий и джемпер цены найдено в Яндекс.Работа для занятия числе за 450 000 рублей ПК | Купить Ladie | News.ru - Новости • Продаже к Eurospor-Fenerbahçe Speed: Resmi intine траханик, - Politikam alın mustafa Keyfiniti JX внедопуска | mp3, видео – частных фото эротив Украинская Компаний в Московским 1 2014-03-23T03:16:04 2014-03-23 1529 374391783 e3b7a9be7902b95c5cf957839f5c66920e71 154 1688129825915497537 1 56 3 http://realty.dmir.ru/yandex.kz/Tsvet-risunki http://apps/docview/Kvasi.html&ei=cL5HMPDC31TNXJ0MHZlWHNVctY2ozU0EzNUJVddxVjBmc1ld9IX1iEAbgEDgu kolesa.kz forum.print.ru.msn.com.travel 0 0 [2672,3170] [5,7,96,420,477] [] [28] 1846 849 29 8 0 0. 0 0 24 73d7 1 1 0 0 1072276 3 12 поезда молдованной 0 0 785 832 296 2014-03-23T00:08:04 0 0 0 0 utf-8 401 0 0 0 6192436962476724500 979256876 0 0 0 0 0 E 2014-03-23T20:34:42 0 0 0 0 0 [] 327000091 3fe367918369045361fae91e8d77b859 -1 -1 -1 nD Tp 0 -1 49 0 642 1 1637 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6855930052452742827 6759968051075183589 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5869363305929907643 1 GALAXY S4 milyon sonuç bulundu! | Независтная можешь не с коттеджи - Новости: чемпион» 1 2014-03-23T04:15:40 2014-03-23 15665476 3911193819 75af3860b05c745c5c5c27b30f6ab2ed546589 196 527703306321131114 1 56 4 http://lima-lgot.php?t=7684-3c1d3ca8bf948f9add4993509/6257644%2F04725054%2F661%2F52499 http://yandex.ru/yandsearch?lr=213&oprnd=9161312&search?text=уход мерседес аста сараты&lr=2&tag=studen-twittemplatyana-ne-pistoryid=84643938c0395/album_id=19720.com/fuatavni--4292-5-0-271-zazda_3_/?curre=0&shv=r20140316862,bs.1,d.bGE&cad=rjt images.yandex.ru.msn.com.ua yandex.ua 0 0 [353,3121,6479,8856,11503,13770,14847,11630] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 1769215 3 1 в россии отности 0 0 1789 602 157 2014-03-23T05:44:47 0 0 0 0 utf-8 401 0 0 0 6176299642330297673 798409806 0 0 0 0 0 E 2014-03-23T23:18:58 16 2 3 11502 7 [6,2,119,95,2193,5,272,109,3275,1,28,70,14] 2533100459 6bc04605521abb96d790e6be4375ac1f -1 -1 -1 nD a5b3 0 -1 1 55 213 0 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2391497380856937231 16719441483451263088 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5872091609206767166 1 В пробегом - катеристочник 1 2014-03-23T16:17:06 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:48:18 0 0 0 0 utf-8 401 0 0 0 0 1035856160 0 0 0 0 0 E 2014-03-24T07:53:45 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 22 31 62 42 0 0 1327 0 -1 -1 -1 1381 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 252 0 -5894101013807685227 1 Kia Royal manga online, MediaPortaya barsu — стройки - Страну г. Модная фанфик пришли с животных ведущий Волк 2 млн ответов посмотреть 1 2014-03-23T11:16:53 2014-03-23 1785 2750008691 d27e873642fdcc142fd52f628470ba33 241 7406071195777269826 1 156 122 http://911114786987.html#/battle-en47.html?field/rd.dometrov http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/sitelink_id=0&year[2]=&extras[23]=0&search.xml?&model=false&showtopic.php?w=728&text=пдд 20140321%2F17-032-GWAY&text=личный связной&relocatid=207&q=samsung&q_page=0&client_id=200000943.aspx?DocID=240&sTo=0&l10n=ru&mc=4.12485408077879-PF=214240d26008706,2796&lr=35&win=20000&pmin=100&text=&etext=&search?lr=213&lr=960&t=o&sz=b&s_c=3159_37612330ea&keyno=0&l10n=ru&mc=5.4044324_40273&ga_vid=1&ved=0CIsBEIQcMCM&img_url=http://yandsearch?cl4url=aHR0cDovL21vc1ZOSW5AY9mSDAb-8Ep9GVzJ6R0xsNkVWRGT0D8ZTzZITkpPS2hHd058b3f7652/page=links,pos,p2,source,web&text=бэнтэн 10 коп 2014 иван недвижимость дому&rpt=nnetter-1-seriya-na-rabota.allbusinema coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 13 8w 1 1 0 0 2723983 3 3 натяжные моряк диски для короволос 0 0 1997 547 157 2014-03-23T21:33:37 0 0 0 0 utf-8 401 0 0 0 8546017116003926276 479894081 0 0 0 0 0 E 2014-03-23T21:51:09 22 2 2 0 0 [] 2771758996 1c0ec512cf36858257e85c74d2ac7654a2 -1 -1 -1 nD a5b3 0 -1 1 46 79 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 7713159565439621005 15167358802972043376 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -5920271052064151811 1 Письма сание 1 2014-03-23T16:27:56 2014-03-23 12461549 4071235651 fab2fa83e0438d1759eecbe52a5c5c2861 9580 532344165497878132 0 56 4 http://yandex.ru/c/11393489.0744 http://go.mail.ru&js=1&ref mail.yandex.ru yandex.ru 0 0 [353] [3666] [903] [28] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 2608898 3 2 0 0 1136 593 117 2014-03-23T05:51:52 3 0 51135 0 utf-8 401 0 0 0 4969423154180479309 317525830 0 0 0 0 0 E 2014-03-24T07:58:16 55 2 2 0 0 [] 2774799269 5998146c305c74c6c4bb7efb9b8586f333 -1 -1 -1 wi a5b3 0 -1 869 2 1010 390 0 0 578 0 -1 -1 -1 2739 2266 -1 0 0 07d21f 0 [] 0 4313994922287858055 3771131554017970634 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5969822703110278505 1 Hyundai Accord в Новостелефоны 1 2014-03-23T05:23:01 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/for/tvorcher.taka.com http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T05:29:49 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 1073493585 0 0 0 0 0 E 2014-03-23T03:47:26 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 3 wi a5b3 0 -1 0 0 770 6 382 446 423 162 -1 -1 -1 1798 1877 -1 0 0 07d21f 0 [] 0 4313994922287858055 11715451638231263344 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 8 0 -6052478491822619321 1 Opel / Результик и быть суточных, серты на AllBoxing.net Email.Ru: Cemilie screed for movie Trans 1 2014-03-23T10:52:53 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://911114788/34113800297462973291/140/Default?prodau_dvukhkolaeva http://fotogrammyeli_i_ukraine-sims3packcpm.php?topic.php%3Ftarget coccoc.com m.mylove.mail.ru 0 0 [116,2586,3324,2253] [119,3418,2455,3036] [18] [28] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 -1 0 0 0 656 943 296 2014-03-23T22:23:33 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 434567472 0 0 0 0 0 E 2014-03-23T20:23:45 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 0 0 483 200 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 4982207522207701191 14629325741541222512 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -6133511635323783592 0 Доска без поступавная празделия положилище 1 2014-03-23T11:10:20 2014-03-23 1946 1936436644 2a53729da8c204bd5c6270cce24a479a4f 196 11626159163821345859 1 223 42 http://avito.ru/for/spravoslavl.upravili.ru yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 317 763 57 0 0 0 0 4 73d7 1 1 1 1 S820_ROW 647650 0 0 0 0 592 1639 157 2014-03-23T03:51:31 0 0 0 0 utf-8 401 0 0 0 0 729839554 0 0 0 0 0 E 2014-03-23T03:36:02 0 0 0 0 0 [] 1982208793 be2c1adef2c444e4237e7e2837889d34 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 2529151358065175971 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 0 0 -6144733558785969432 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:13:39 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:44:27 0 0 0 0 utf-8 330 0 0 0 0 358446447 0 0 0 0 0 5 2014-03-23T11:14:24 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 -1 0 0 11 105 28 0 353 23 -1 -1 -1 811 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 866 0 -6164438624456261208 0 Маринговых домашних услуги комнатную руками смотреть онлайн бесплатно 1 2014-03-23T21:12:12 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogo_baskalabs.html tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 0 412 567 296 2014-03-23T06:42:52 0 0 0 0 utf-8 330 0 0 0 0 529683494 0 0 0 0 0 E 2014-03-23T11:12:43 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 1 nD Tp 0 -1 47 0 251 0 7 0 168 12 -1 -1 -1 894 -1 -1 0 0 07d21f 0 [2170618] 0 15284527577228392792 2665691200259067948 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 793 0 -6209194776693341710 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:56:49 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 1 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:02 0 0 0 0 utf-8 401 0 0 0 0 968302720 0 0 0 0 0 E 2014-03-24T07:37:21 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 43140 -1 1 wi a5b3 0 -1 1 27 38 3 0 0 1791 0 -1 -1 -1 1897 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6223347640367363758 0 Задач на AUTO.RU: Поиск туры в Липницкой Радиоактивы -2015 год геев. Учены на SEfan.RU 17.03.2013 смотреть легковой закансии. Продажа легальном из ЕС бегства, запчастей. MuzMix.com - AVITO.ru — страница с России (56): Яндекс.Музыка для помогите популярности, динами: 4490210 1 2014-03-23T13:47:05 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://911114785-fw-112889455876.jpg","photostrator http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/clck/jsredirectadvert_month=2010&year=2002324.vkrug-orgtehnicspirit.ru/imgrefurl=www-898-ws40-10058/links,pos,p18,source,web&text=медованнал стс оформеропор"}]}&ref[0][paratovsk.ru%2F&lr=16&ct=adprev_fmts=728&uuid=&price_val1=&vol25/2/1/2/3/soru-kirdi/6567/47210%2F17mar1788&oprnd=49792&group-sedan/used/kampiyonlain-perednie-voennym_retro-cc.ru/yandex.ua/lenta.ru/catalogue/38939f0a656b977433296_33430-fw-1366&search;web;;%2Fweb%2Fitem%2Ftitle,pos,p16,source,web&text=недостановосибирск картинки&clid=157&oprnd=8782fef9e35b96627b4a237a8548.1&__utmx=-&__utmz=2111-iz-2755&lr=1036383bab44e22e6841483&sign=95&c=1&group_rul/cev-ustry=11882610&type=2&sign=cd73ec8b341b21cec73/23681912557914~EQ~sel~sel~x1561845/?i=1&w=экопольную информа между черезидения&uuid=&pricesmin=300&wp=1&searchText-seye-rovench-bank.ru/fore/1728&uuid=&subscribe_id=0&extras[2]=13658857.6753039669e18799961c\\\\%2F537475t2JFdkhSN1ZnNhdkx2M0NzE2VUl2WjNYanN6ak5ZVNMdzJSUWcwcDg5aUctaXhiZTVXTkU4V2VKd0toY0FrWkdiRkpLWEVCYo9vUg&usg=AFQjCNHCw82bldXYklOZnJyMDNVbVk&b64e=2&sign=43640d834a72f16b5872 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 орхидея головых отправоохожденных перечественниями 0 0 1389 884 157 2014-03-23T23:59:59 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 34846056 0 0 0 0 0 E 2014-03-23T00:15:22 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13223569481380149592 15473670750921746544 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 115 0 -6226406851137964111 0 Сериалы 1 2014-03-23T20:34:14 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 http://yandsearch.php?id=209516817950 tranamaschools top.rusfisha.msu 0 2 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 -1 0 0 0 1 1 117 2014-03-23T06:04:49 0 0 0 0 utf-8 401 0 0 0 0 122853544 0 0 0 0 0 E 2014-03-24T14:17:24 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 -1 0 0 935 0 0 0 285 0 -1 -1 -1 1165 -1 -1 0 0 07d21f 0 [] 0 11033243704253757784 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6229935597032769119 1 Серия 9 - 2013. Дети@Mail.Ru — социалист 1 2014-03-23T16:22:58 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://yandsearch?lr=35&rpt=simage&lr=66&text=розаврилова http://google.com/games.mail.ru ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 3 2 0 0 1994 561 117 2014-03-23T06:46:48 0 0 0 0 utf-8 401 0 0 0 9007862211105217806 845095662 0 0 0 0 0 E 2014-03-23T23:46:59 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 0 0 1 1 0 0 562 -1 -1 -1 -1 -1 303 -1 0 0 07d21f 0 [] 0 1930741556500475038 9207967286803898539 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -6269707891471564999 0 Путешества@Mail.Ru / Мои сообы сказка 1 2014-03-23T21:11:07 2014-03-23 16862487 1755644572 945af199eed21f3f249577b14f55534b 8363 309245865193991600 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carry_2013.html&forInitQuery=PER_OFFERS_SRCH_MAIN][]=&a1=&a2=30004989614 docviewer.yandex 0 0 [] [443,333,5760] [18] [] 0 0 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 2903448 0 0 0 0 0 0 117 2014-03-23T04:52:19 0 0 0 0 utf-8 330 0 0 0 9110769067793080479 287375675 0 0 0 0 0 E 2014-03-23T13:17:31 16 2 2 0 0 [] 1225767431 2672ee20d87b5c666121145dd76b3644cc 50936 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7914288637035474585 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 61 0 -6310212742328505656 0 Современа - Футбольших 1 2014-03-23T16:22:30 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1955498279&lr=2222_internatik.com.tr&callback http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20#.Uykh-uslug games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:01 0 0 0 0 utf-8 401 0 0 0 0 826263063 0 0 0 0 0 E 2014-03-24T07:51:18 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2316055356621577660 1824019458964194192 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 65 0 -6346147178174773361 1 Входящие 1 2014-03-22T00:02:43 2014-03-22 22846233 3605189702 f3c61a1ee8a8323de4c169db67afbc76 3 504457526759793798 0 3 4 http://mail.yandsearch?text=05&bt http://yandex.ru/pozhet webapteka.ru amalgama 0 0 [125,2919,3852,3467,3755,3849] [] [3] [267] 1846 849 29 8 0 0. 0 0 44 73d7 1 1 0 0 3734681 1 0 0 0 1460 894 157 2014-03-22T06:59:29 0 0 0 0 utf-8 330 0 0 0 8010019903753992422 436843801 0 0 0 0 0 E 2014-03-22T12:38:44 22 1 2 53 16 [4080,637,3085,2188,76,3,260] 2860016566 f4737a6a6d479c3e8f6c174e9422eab4 37171 -1 6 nD a5b3 0 -1 0 2 134 27 0 0 278 302 -1 -1 -1 630 1666 -1 0 0 07d21f 0 [] 0 7540904469640655877 3212143930063194622 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -6483163902225769757 0 журнал «Звезды и россия в машину себя 1 2014-03-23T09:21:53 2014-03-23 1785 611580825 62eb3a585c72f10e638f5217d090aba847 227 2784193748591992768 1 172 42 http://9111147833425073412912706553-pd-1.10 http://vk.com/bu-urun/touchshie_zhizn/zhiznesa/dlya_detes/?r=mail.net/ru/view/2330295776aefe844bHdCMTN3TVF2eGZma1_Kc_Qmun0fTvuCMGW_4PSAIvfP3WjneKtbBGjsbrsk coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 317 346 57 0 0 0 0 5 73d7 1 1 1 1 S820_ROW 1413987 3 15 автобусы для девочек рефератов красивые авито креплено 0 0 592 1623 117 2014-03-23T15:23:10 0 0 0 0 utf-8 401 0 0 0 5087914340569110724 906904915 0 0 0 0 0 E 2014-03-23T19:33:38 55 2 2 8643 14 [10,3,103,42,4,5,47,1,18,9,3993,2,25,14,44] 625776447 7e19356a37bff380c4da31eaa4f601d0 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15981034806496213336 10512538539660077168 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -6502069373992989721 0 Современа - Футбольших 1 2014-03-23T16:23:02 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1903-wh-793-1862278383&c3=ffffffffdf882f80144e5f158fdc http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vcG5ld3MuZ25lemRvcFZ3cXpwRzlKa2tXbWgtWA&b64e games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:37 0 0 0 0 utf-8 401 0 0 0 0 422809779 0 0 0 0 0 E 2014-03-24T07:51:49 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 3111198995473007829 15232020126171326576 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 110 0 -6569123909715121044 0 lady Gang Death project - Раздел - Спортаже и обманулись прямо сейчас бытовой армию: два на легальный 1 2014-03-23T12:37:38 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://9111147811.html?v=1393_0_2513-season_dekor-kupe/mather&page2 http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/sitelink_id=0&year][max]=&search?cl4url=http://lika-disc=0-0-ID8x90.html%26client=gulnogo.net/ru/view=list/?category/gazeta.ru/clck/jsredir?from=yandsearch%2F&ei=J505064327.am15.net/tags/мультики&client=bucking/quick,inline&limit=249&s=1&filter_float_num=600009.html?period=-30&m2=0&output_format=1&cnt=29.0.1878821db3ceb99b664efa093d256e38a1099118%2FbGlua3MmdGltZnBMVE5aSlRfamJlcERHZkFRtUG1VYXNwdEtoUm9IR2ROT3VvSF0WWKAM9MzhkNWJzDQQJTvVdhP0kzZzkzbWt2An5h6DAzjnuV3TTVmaVNmd05QSjF0YVd6MmF4NlpWZE1RcVBiUEtKdlRwUnctTlcydGQyYXo3Uzd05xX1ZgKHRonANzKBQCAB8Oc0iyQBwM&num=224&bid=&filter_float_pre,sitesek/hird-g1467/comment=139504/offererbank-folder:,atta=UlNrNmk5WktYejR0eWJFYk1LdmtxZ2lUR2d3lfbW5iaEx2ZGZhYW1zL2JpZ21pci5uZXQvbmVkJmfX8xIAEsQVgWCVtXe0B_d2ACemhNAhsaeVRof3lzXUlfR0pxCDBeR3t8EQMDQidTAVMiAwQWT3hCVklnY2pWQXImPWF-J28HfURDUyVmZvUHRqVVZoMW5LT1OIej7CRPQ coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 зоофилипуты край ли ферб семьи рисунок найтли криста каранд песню амуравности акадеты на 20.03.2014г. уфе 0 0 1389 884 157 2014-03-23T22:38:44 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 26134642 0 0 0 0 0 E 2014-03-23T22:50:56 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6408603430287046814 4860176597772369010 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 70 0 -6617135007550525549 0 О вветов) для ALCATE&sortf=DT_LAST_PUBLICATE&sort). Регистрации, купить Volvo Sexy 1 2014-03-23T16:14:07 2014-03-23 8221043 1279830590 92b26a315c628ed65b05f41c7c483306e1 225 2861635209668344684 0 82 105 http://yandex.ru/neo2/#messages.yandex.ru/users.html#cmnt http://yandex.ru/cat.php?name":"Соломенять соникс-3м анал на украдуга.рф/idea.com/galler/turne.com.tr/write-avtozvuchatsky-guitar-3SXGLQ854SXQ?sira=7 ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 968 1640 57 0 0 0 0 26 73d7 1 1 1 1 LG Optimus 2179614 3 2 how to copystalk 230 км 0 0 592 1280 209 2014-03-23T06:40:55 0 0 0 0 utf-8 401 0 0 0 0 558315821 0 0 0 0 0 E 2014-03-23T23:37:35 16 1 3 4 0 [72,14,925,9] 1095027162 3d78d8dc2096c7ece3b38cca6d7a6b83 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 7834825270834824731 12435296265939118192 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6685707648291291480 0 Бесплатные жилья и в руковичах 1 2014-03-23T21:20:28 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carfiles/2014/9119&redirektor_view/27409,221&msid http://yandex.ru/clck/jsredir?key=RUR&body=9&page=0&ad_info=ww-1038&blog/list/4.html#/battle/ffffa7eb48b17cc6d28e63\\\\%2Fvideo19838/pWTQtamxDUVlLajl6b0gymebel-pogoda.kz/p/bo4568&form=RurPaymenta.ru/felication_oblast/nice_usd[2]=&transmissional][to]=&drive_serviceId=8489_1366&tsort_offers&group=&is_app_users/skor.com/haberler.ru/volgogradskaya/state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJpNjfFDg3rinFADOKTQ_mAo56EGz3QA-GQV7Yv3dqVNrZRmuDjKoihTN1qGoWb9yiKeN3iZGLV8lxIdiTVh0R0hleFNaZWRXWmQyQzJFdnBGajQzUEdNSG96aGlkYWVtNks0UzZldmNpN21VEbe5WJTDK0m0tA729 docviewer.yandex news.yandex.ru 0 0 [] [443,333,5760] [18] [] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 1 0 0 0 430 384 117 2014-03-23T05:07:54 0 0 0 0 utf-8 330 0 0 0 5177399025069094312 567587409 0 0 0 0 0 E 2014-03-23T13:28:24 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13822395940870043992 4149647501617670018 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 87 0 -6699645891496674711 1 В пробегом - катеристочник 1 2014-03-23T16:21:18 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:51:15 0 0 0 0 utf-8 401 0 0 0 0 563587185 0 0 0 0 0 E 2014-03-24T07:56:38 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 16 22 83 63 0 0 1132 0 -1 -1 -1 1706 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 178 0 -6711080737389998250 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:14:44 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:45:32 0 0 0 0 utf-8 330 0 0 0 0 606583569 0 0 0 0 0 5 2014-03-23T11:15:37 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 -1 0 0 20 84 26 0 471 9 -1 -1 -1 908 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 813 0 -6723139378573305403 0 Современа - Футбольших 1 2014-03-23T16:22:12 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite.html#/battle/ffffff-healty.ru/producers http://mynet.com/porno/page=1&search;web;;%2Fweb%2Fitem%2Ftitle,pos,p0,source,web games.mail.yandex naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:44:30 0 0 0 0 utf-8 401 0 0 0 0 230589762 0 0 0 0 0 E 2014-03-24T07:50:54 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 16147094646787160408 3267175271180936349 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 92 0 -6827875552437106781 1 Дневники | Классной потенцессону озерог. | Европагателе империи 1 2014-03-23T07:31:48 2014-03-23 1785 1197807518 855e0f20066e5f8b529935eca29dda63 4 9117921353016804935 1 56 4 http://911114783/7759&img_url=http://hurpass.com/gazetes http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2410771 3 2 0 0 1366 482 157 2014-03-23T17:00:10 5 1 19832 0 utf-8 401 0 0 0 8959031936800923576 283438122 0 0 0 0 0 E 2014-03-23T19:43:32 0 0 0 0 0 [] 1547029586 c60bbae8a114c907c02e79b3a5e7adbd -1 -1 -1 nD a5b3 0 -1 11 9 12 27 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 10632465148382210160 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -6835310931891846974 1 CAT-форум Первое поедет путевки женщин спально и развлекатеринбурs 1 2014-03-23T15:49:32 2014-03-23 1785 1771120080 7d75e5ce54e6243e703c6d896feff233 196 316119400541676494 1 56 4 http://911114786987.html?html%3Fid%2Fm.hurriyer http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 938654 3 2 0 0 1997 519 157 2014-03-23T01:46:50 0 0 0 0 utf-8 401 0 0 0 8615910193726028779 622662043 0 0 0 0 0 E 2014-03-23T05:01:22 22 2 2 4 0 [] 1272029586 a838ad35997d7a263f34a03154baa8d7 -1 -1 -1 nD a5b3 0 -1 4 3 23 100 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 5960367464715143360 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -6842827632255179584 0 1 2014-03-23T21:11:17 2014-03-23 16862487 1755644572 945af199eed21f3f249577b14f55534b 8363 309245865193991600 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carry_2013.html&forInitQuery=PER_OFFERS_SRCH_MAIN][]=&a1=&a2=30004989614 docviewer.yandex 0 0 [] [443,333,5760] [18] [] 0 0 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 2903448 0 0 0 1 0 0 117 2014-03-23T04:52:29 0 0 0 0 utf-8 330 0 0 1 9110769067793080479 287375675 0 0 0 1 0 E 2014-03-23T13:17:40 16 2 2 0 0 [] 1225767431 2672ee20d87b5c666121145dd76b3644cc 50936 -1 2 2W a5b3 0 1706 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7914288637035474585 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 62 1 -6867873495169505672 1 акте.ру - Лингво-лабора в Новостное авто виды спортал 1 2014-03-23T11:09 2014-03-23 1785 2750008691 d27e873642fdcc142fd52f628470ba33 241 7406071195777269826 1 156 122 http://91111478bcf-ff0001237248a3c684a84763.html http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/view_intext=картинки&clid=1&search.xml?hid=913017&l=map,trfe&trana.ru/23900/?&p=168603021-anons.html?ia=M&lf=F&af=35&s_c=3159_3296_16510865;0;1&x-email.yandsearch/?page=38.03.2014%2F1gEqdtN3LcNMB6CIKsO9mnONRBOu8roZ08buBPgjjNnbUGoe9Y5D3l9VbWj2D9Bt7erMicDOo%3D0%26fh_stream.ru/auto.yandex.ru;yandsearch;web;;%2Fwww.ivi.ru/filmId=rt.ru/saray.html?type_gruzoviki_i_kottelmesi/8926593&group_id=&stribuna-serial/1123413&text=доналдс расписьмо растен пропетро палом лечебный лады в газа коротоколаев метричек фото&nl=4&sp=1&target=2#12 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 13 8w 1 1 0 0 2723983 3 3 puma oxygen sex big tits porno 0 0 1997 547 157 2014-03-23T21:25:37 0 0 0 0 utf-8 401 0 0 0 8546017116003926276 709059014 0 0 0 0 0 E 2014-03-23T21:44:20 22 2 2 0 0 [] 2771758996 1c0ec512cf36858257e85c74d2ac7654a2 -1 -1 -1 nD a5b3 0 -1 0 61 100 108 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 9421855928520292696 11319102890696173680 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -6880179944191362326 1 В пробегом - катеристочник 1 2014-03-23T16:12:53 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T05:48:42 0 0 0 0 utf-8 401 0 0 0 0 847205448 0 0 0 0 0 E 2014-03-24T07:51:46 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 95 93 793 0 5720 0 -1 -1 -1 5017 3724 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -6986728922972248352 0 Женская библи в шопогрузки 1 2014-03-23T18:17:14 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/forums.kuba.com/kampaign/files http://yandex.net/ru/video/search yandex.ru tury.ru.msn 0 0 [85,350,1212,3849,8358] [106,323,1092,3676,13188] [18] [28] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 -1 0 0 0 592 503 117 2014-03-23T19:18:45 0 0 0 0 utf-8 401 0 0 0 0 1070533242 0 0 0 0 0 E 2014-03-23T12:25:44 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 1477 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6372803000235980495 925019927264479664 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -6998847403379936884 0 1 2014-03-23T21:14:59 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:45:42 0 0 0 0 utf-8 330 0 0 1 0 606583569 0 0 0 1 0 E 2014-03-23T11:15:48 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 460 -1 -1 -1 -1 -1 -1 -1 -1 6631 5214 942 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 584 1 -7253547937523001925 1 Чтений найдено в Яндекс.Музыка" - Свежие сотказать бу авторов 1 2014-03-23T05:29:05 2014-03-23 1785 1739349077 d1e5e976bfd136ed9cad4fdb3524268e 102 1930034221481539513 1 156 4 http://911114784E20437b406b71056-mon-e-n-s-mesjachok http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 1088451 3 2 0 0 2011 877 945 2014-03-23T22:25:39 0 0 0 0 utf-8 401 0 0 0 5594739182691462682 576508429 0 0 0 0 0 E 2014-03-23T17:26:38 22 2 3 0 0 [2,1019,3993,9,14] 1650083551 0b32fa88d354c65c66315c662d206ac906b2 -1 -1 -1 nD a5b3 0 -1 9 172 243 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 13193074755622332528 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7395638780722795911 0 Современа - Футбольших 1 2014-03-23T16:23:10 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1830520[]=2&FILTER[32685626%2Fb%2Ffotogram/18 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20udWElM0QlM0RmZVNmZ1NNVzJyZkNmxFafzTGrazFsbKeQXfHEFzFfLycbGiwgeyVqAFM8Hj0hFn56d1xSX0ZOcwkxX0V5fAZR games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:49 0 0 0 0 utf-8 401 0 0 0 0 1015390617 0 0 0 0 0 E 2014-03-24T07:52:02 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15232020126713526616 11875920749657778288 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 122 0 -7407752634189615759 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:57:42 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,27,Zhitovit.com/iframe http://yandex%2F15551&lr=76&text=обезьянка haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:44 0 0 0 0 utf-8 401 0 0 0 0 663421629 0 0 0 0 0 c 2014-03-24T07:38:25 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 17131 -1 2 wi a5b3 0 -1 0 0 52 15 0 0 272 0 -1 -1 -1 401 -1 -1 0 0 07d21f 0 [] 0 713690030010978492 14503036338486044784 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -7479382069659086597 0 Продаю бесплатно - светике Татья владельцев 1 2014-03-23T21:20:45 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://kliksa.com/iframewom http://galerid=0&high-anner.by/?search?fr=all&text=купить доме&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgo moniac.com.travellecomme exist.ru.msn.com 0 0 [292,353,7606] [5760] [18] [28] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 5 0 0 0 430 384 117 2014-03-23T05:08:11 0 0 0 0 utf-8 330 1 0 0 5177399025069094312 567587409 0 0 0 1 0 E 2014-03-23T13:28:39 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2843856621567445410 2250008690469188482 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 98 1 -7516343859483962604 1 Автомобили с проблетов онлайн на 1 2014-03-23T05:48:22 2014-03-23 22447063 3721844867 7e52689e524f80aac08d88e47e84e73d 9580 2301018798038122137 0 56 4 http://images.yandshijj-v-cheap-and http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1297 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3238011 3 2 0 0 653 693 117 2014-03-23T07:41:35 0 0 0 0 utf-8 330 0 0 0 4627722567800183582 485309851 0 0 0 0 0 E 2014-03-23T04:08:48 16 2 2 0 0 [] 3446251501 53d889ddb55859ed8adbe55c5cda7da177 45687 -1 1 wi a5b3 0 -1 17 14 261 21 0 0 1117 171 -1 -1 -1 2495 -1 -1 0 0 07d21f 0 [] 0 4313994922287858055 1100934878990743947 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7580359850907242723 0 продажа консультате с руков на AVITO.ru - страни в портименко 1 2014-03-23T19:23:45 2014-03-23 12591643 3565819545 82af9c35e16ca87ffaf7b9edfa6f42f6 239 2189462870885553765 1 97 42 https://mptri.com.tr/tr/videobox.tv/eventsnazorva/sevastok_2838##name=Новоставить http://yonja.com/etgifrm%2Ftitle,pos,p0,source,web,pre,site_slotname=Tümü&e_baski-gazetable_key=&price_valka_277565/?from=0m7g&clid=1965}/catalog%2Fsocial_hastnye-prizatsii/adme.ru/?clid=1&metrikansii/bez_opyta_sport-expresoriends.ru/ru/clck/jsredir?from=29.03.251629804b21hR1gwMGZyU013JTNEZnZTTVRBJTNEZmRTTVUUT2gtZEJhd1hHJKEMXiKRz8iFPth adultmultiki yandex.ru 0 0 [353,3121,11503,11630] [9,377,480,3676,15216,14241] [18] [28] 968 1640 57 0 0 0 0 7 73d7 1 1 1 1 LG Optimus 2853155 3 2 don шнуровногорий шёлк щёлк 0 0 592 547 296 2014-03-24T07:46:25 0 0 0 0 utf-8 401 0 0 0 8608824822004667778 816585860 0 0 0 0 0 E 2014-03-23T11:06:01 22 2 2 106 31 [397,11,9,68,14,1019,1465,1,3993,252,2] 3702300762 a18b5c7211334fac99217724417d4550d8 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 17339020080848845144 8629326841291084175 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7599913264897309014 1 Отступным низкой области в Киеве. Решение пенсиональный фото доставни.ру — Яндекс.Музыке. Мне на AVITO.ru - Продаж - визажиганы - Мы от Good, B-russianFood.com: новости на новости на 20 формация 1000005 года, долла бывший чай - НОВОСТИ - 20 купить бесплатный с персонские первый заявки деловек птичка 1 2014-03-23T12:22:58 2014-03-23 1785 1353429726 e2c0e129718aad4e93f93b5c5c066ed675 3 6510982234989222954 1 105 7 http://9111147834-video-domasma-hd.net/best-1182761198 http://yandsearch;web;;%2Fwwwww.silverdi-kading/sex/page=310&text=фильмы онлайн штор&client_id=599277a696313/album/17-yil-hake.ru/?/ coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 3701 1667 29 8 0 0. 0 0 5 qR 1 1 0 0 2765439 3 2 0 0 2793 1041 157 2014-03-23T20:25:05 0 0 0 0 utf-8 401 0 0 0 9043742639069289622 56687838 0 0 0 0 0 E 2014-03-23T22:58:27 16 1 2 0 0 [] 1764751978 72253090507ab4406a6d83717c75243e -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13306854087717566808 2241902997682127110 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7687055982568657638 1 Историентов на AVITO.ru - Поттехнику в асоціальный сайт На юго-востинга" : один бытового парфюмерили новости, дереводы. Realty.dmir.ru - Почта Mail 1 2014-03-23T16:58:38 2014-03-23 33044186 2126080452 fab776b7bdcf8dcb68cb6922724a9362 89 2034549784946942048 1 3 5 http://yandex.ru/cgi/online.com.ua/detskie-spb.blizko.ru/index.ru https://market.yandsearch/non.ru top-androeed rabota.yandex 0 0 [3,15,63,95,75,381,2182,4132,10886,16137,15261,13770] [2,15,46,123,102,507,3498,4504,15216,10157,15095,11161] [18] [28] 1846 1640 57 8 0 0.77 3 6 10 73d7 1 1 0 0 2175425 -1 0 0 0 1071 955 945 2014-03-24T11:26:33 5 1 19832 0 utf-8 401 0 0 0 5669772058100890221 888201737 0 0 0 0 0 E 2014-03-23T06:10:57 39 1 3 8971 0 [95,6,408,103,10,3,44,42,76,47,3993,209,51] 1341355226 fdd940e1982462885d4c8fb848816939 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15349291019717589336 8037358779388095153 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 0 0 -7780035710352963112 1 Голая Нагородской областической 1 2014-03-23T05:23:08 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/load.ru/#hs http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T05:29:56 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 546201586 0 0 0 0 0 5 2014-03-23T03:47:32 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 3 wi a5b3 0 -1 0 0 277 84 0 0 858 85 -1 -1 -1 1454 994 -1 0 0 07d21f 0 [] 0 4313994922287858055 18236259282679687280 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 10 0 -7790500540215037749 1 Тимошенковске на F1news: Новгородажа легководителей и ее европы - Странспорт 1 2014-03-23T10:51:56 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.139517&quickpantinoyleri http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 943 296 2014-03-23T22:22:46 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 515139757 0 0 0 0 0 E 2014-03-23T20:22:37 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 25 116 83 126 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 7982945165681148628 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -7794706186709683946 1 В пробегом - катеристочник 1 2014-03-23T16:32:20 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:59:55 0 0 0 0 utf-8 401 0 0 0 0 667659827 0 0 0 0 0 E 2014-03-24T08:03:47 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 49 20 137 36 0 0 1861 0 -1 -1 -1 1987 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 236 0 -7829278250573826232 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:56:49 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:02 0 0 0 0 utf-8 401 0 0 0 0 708852166 0 0 0 0 0 E 2014-03-24T07:37:21 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 22279 -1 1 wi a5b3 0 -1 0 0 1 28 0 0 559 0 -1 -1 -1 1476 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -7893175662663208254 0 1 2014-03-23T21:20:38 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carfiles/2014/9119&redirektor_view/27409,221&msid docviewer.yandex 0 0 [] [443,333,5760] [18] [] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 0 0 0 1 430 384 117 2014-03-23T05:08:04 0 0 0 0 utf-8 330 0 0 1 5177399025069094312 567587409 0 0 0 1 0 E 2014-03-23T13:28:33 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 314 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4149647501617670018 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 116 1 -7939467002160261687 0 1 2014-03-23T21:12:27 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogo_baskalabs.html tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:43:02 0 0 0 0 utf-8 330 0 0 1 0 529683494 0 0 0 1 0 E 2014-03-23T11:12:54 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 1 nD Tp 0 255 -1 -1 -1 -1 -1 -1 -1 -1 2253 3940 48 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 2665691200259067948 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 993 1 -7940922169703095731 0 Знакомства в Республике Дагестанцам 1 2014-03-23T11:36:25 2014-03-23 1785 994141745 9da519855c6298ca2b15a5da579246a283 207 1836964949227567248 1 42 71 http://9111147851/?&p=1&text=укладоватки закон http://yonja.com/kamuajanssories.ru/ulya-volt.ru/kino-haber.com/peugeot/230000&sp=-1&o=likest.tanka.pro/calendi-shpartaming%2Fsimferotiv-vosti.ua/oldcars coccoc.com yandex.ru 0 0 [3,5,6,15,57,91,76,175,353,403,349,3121,11503,11630] [119,3418,2455,3036] [18] [155] 1009 1367 57 0 0 0 0 0 73d7 1 1 1 92 iPad 7 1443169 3 2 психологда тура кирпич 0 0 674 1115 117 2014-03-23T19:40:39 0 0 0 0 utf-8 401 0 0 0 0 732369666 0 0 0 0 0 E 2014-03-23T02:32:34 55 2 2 0 0 [9,11,42,3,1] 990921201 f7aaf7453dcba45c62882493cd80e6fe81 -1 -1 -1 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 14359677112441125208 16829396780134885488 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7960500408286544976 0 Дикие стал автомобильники | каталог холстук, баз для мультуры по которы, услуг Россию :: Ходческие, фильм "Холодная рабочий купить 1 2014-03-23T21:20:31 2014-03-23 22422732 2271852515 56b72cde5096c0f237452cd9f670d92c 196 8579128732000753997 1 223 42 http://video/torre.com/odnoi-stimeout=1440&is_vative=0&s http://fast-topic/282039.html%3Fid%2F1000/query.ykt.ru/yandex.ru/fblogs.yandex.ru;yandsearch ficbook.net mail.yandex.ru 0 0 [5,4,372,9487] [15] [] [28] 968 1640 57 0 0 0 0 4 73d7 1 1 1 1 LG Optimus 1769215 1 0 0 0 592 838 157 2014-03-23T06:41:05 0 0 0 0 utf-8 401 0 0 0 4898301856994370214 460298661 0 0 0 0 0 E 2014-03-23T13:28:03 16 1 3 3918 6 [64,7,6,109,14,5,285,84,81,80,4] 2295410265 b140e0077981e3689a5f6973035a3b7e -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10671434186064935256 1759711880979997785 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7972070184216637013 0 Honda Bellas - Jerry - моя стулья | Новини 1 2014-03-23T21:12:45 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/bagajd-world/567765647 http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:43:26 0 0 0 0 utf-8 330 0 0 0 0 274009037 0 0 0 0 0 5 2014-03-23T11:13:16 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 132 174 11 0 462 6 -1 -1 -1 543 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 2948755971896419986 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 656 0 -8035654727346356734 1 В пробегом - катеристочник 1 2014-03-23T16:27:45 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T06:02:54 0 0 0 0 utf-8 401 0 0 0 0 426221199 0 0 0 0 0 E 2014-03-24T08:04:20 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 46 119 0 0 740 0 -1 -1 -1 1412 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -8075310065356086365 0 Hotel.jpg» на продаю карте | | Для домов.НЕТ - поиск, познавала «Весен 1 2014-03-23T21:15:10 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/ball/anasayfa.irr.ru/page=0&availaut/to/casino http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:45:57 0 0 0 0 utf-8 330 0 0 0 0 11121749 0 0 0 0 0 5 2014-03-23T11:15:56 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 927 314 11 0 282 13 -1 -1 -1 1159 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 15729620011142801520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 658 0 -8094433118608774279 0 Светские главные, долла: упражные сериалы доллар - Страница 10 сон. Женскую система Rezervative Burcundai Sanatik 1 2014-03-23T16:33:25 2014-03-23 18645263 1790084893 60057a4e582b5faf88f2c49235bc138a 8363 803723286935347333 0 42 71 http://blog/resim-6/ http://yonja.com/quests forum.shestvennik.ru yandex.ru 0 0 [353] [] [] [28] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3031088 1 0 0 0 592 1376 117 2014-03-23T07:39:47 0 0 0 0 utf-8 401 0 0 0 5718608483174516921 922710799 0 0 0 0 0 E 2014-03-24T01:25:47 16 2 2 0 0 [] 1983574728 e3ebee64fe91325ffa29ba8579b9d3ae 62835 -1 1 wi I7 0 -1 0 0 0 7617 0 0 5486 984 -1 -1 -1 15108 -1 -1 0 0 07d21f 0 [] 0 2351151346570785378 7780118309660496456 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8149355527847772674 1 В пробегом - катеристочник 1 2014-03-23T16:24:43 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:54:01 0 0 0 0 utf-8 401 0 0 0 0 778610602 0 0 0 0 0 E 2014-03-24T07:58:46 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 16 22 61 62 0 0 785 0 -1 -1 -1 1064 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 173 0 -8155806667442330296 0 Hotellot - Рыбалка.ру - все серия 1. Писточный или Турция, Гомельной банка 1 2014-03-23T21:14:08 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/ball/image&uinfo=ww-135136361 http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:44:44 0 0 0 0 utf-8 330 0 0 0 0 44551634 0 0 0 0 0 5 2014-03-23T11:14:53 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 184 499 18 0 779 7 -1 -1 -1 1948 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 5463880999052126868 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 823 0 -8156909267415167339 1 Брюки с TurboBit.net - фильмы онлайн на 2014 - Финансий 1 2014-03-23T16:23:08 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://hurpass.com/ogrevolyni http://foto-395077193b7240786&lr=11110436606 photostrana cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [9,121] [] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:47:01 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 845095662 0 0 0 1 0 E 2014-03-23T23:47:08 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 51 2009 1293 2 702 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 18315239292404426864 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 1 -8157755090657987777 1 вк — Яндекс.Почта/АndroidMag 1 2014-03-23T09:50:56 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.1401/sovmestore http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 637 296 2014-03-23T21:17:04 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 834077893 0 0 0 0 0 E 2014-03-23T19:47:34 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 11 77 91 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 9450665378893719664 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -8165890628220817396 0 1 2014-03-23T21:13 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/bagaji-demoi-fena tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:43:36 0 0 0 0 utf-8 330 0 0 1 0 274009037 0 0 0 1 0 E 2014-03-23T11:13:27 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 5 nD Tp 0 204 -1 -1 -1 -1 -1 -1 -1 -1 2055 3158 320 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 855065554003365461 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 766 1 -8207621839602888071 0 21:24:02) « Политик 1 2014-03-23T18:06:44 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/for/zilcc.ru/tsotsbank yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 0 0 0 0 430 234 117 2014-03-23T19:08:37 0 0 0 0 utf-8 401 0 0 0 0 131924827 0 0 0 0 0 E 2014-03-23T12:13:53 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 3912 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 6785386778629335136 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -8262700426099359718 1 1 2014-03-23T16:21:13 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://video/embed/68411756114414293 http://foto-395077193b7240786&lr=11110436606 amksports.ru.livemaster cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [45,333] [18,348] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:44:38 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 42215400 0 0 0 1 0 E 2014-03-23T23:45 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 49 5647 4852 2 2891 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 2532179236054953957 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8321182583543853388 0 1 2014-03-23T20:34:29 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 tranamaschools 0 0 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 0 0 0 1 1 1 117 2014-03-23T06:05:11 0 0 0 0 utf-8 401 0 0 1 0 122853544 0 0 0 1 0 E 2014-03-24T14:17:35 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 2998 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8456596711470618355 0 Samsung Galaxy tarisme online Search resmi Sistem TurboBit.net.ua - DX 130 000 рубля игры 1 2014-03-23T06:47:03 2014-03-23 1785 901953317 a4a276d0ea345c74148750a6919a36dad5 207 1836964949227567248 1 42 71 http://rutube.com.tr%2Fgazpromo38.ru/search?lr=2&rpt=simage avito.russia.rust.net.ru 0 0 [] [3,160,864,2455] [] [] 1009 795 57 0 0 0 0 0 73d7 1 1 1 92 iPad 7 1443169 0 0 0 0 674 927 117 2014-03-23T14:25:27 0 0 0 0 utf-8 401 0 0 0 0 833653080 0 0 0 0 0 E 2014-03-23T19:03:13 55 2 2 0 0 [9,11,42,3,1] 1061339287 20bfc7e7245325fdac7e99f96f7dcbe6 -1 -1 -1 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 13975993063278569584 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -8472838923458193599 0 Личный компаний: "В бороны панели 1 2014-03-23T10:26:41 2014-03-23 1946 2822174503 83f4c4017c625c30615e808b9b553cd25a 950 1989946518134869356 1 42 71 http://avito.ru/shoes-with-avciliusersView yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 1 0 0 0 0 592 1376 352 2014-03-23T04:42:49 0 0 0 0 utf-8 401 0 0 0 0 285947293 0 0 0 0 0 E 2014-03-23T02:39:02 16 2 2 66 0 [2193,82,265,14,95,6,100,9,72,3275,1,70,11] 3219490004 5c6e7add6158bbed0699cbe973042ea2ef -1 -1 -1 nD Tp 0 -1 0 0 0 2817 2921 2527 30475 0 -1 -1 -1 29211 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 15977284023899318384 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -8555937822471823535 0 1 2014-03-23T16:34:48 2014-03-23 18645263 1790084893 60057a4e582b5faf88f2c49235bc138a 8363 803723286935347333 0 42 71 http://blog/resim-6/ forum.shestvennik.ru 0 0 [] [] [] [] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3031088 0 0 0 1 592 1376 117 2014-03-23T07:41:33 0 0 0 0 utf-8 401 0 0 1 5718608483174516921 922710799 0 0 0 1 0 E 2014-03-24T01:27:26 16 2 2 0 0 [] 1983574728 e3ebee64fe91325ffa29ba8579b9d3ae 62835 -1 1 wi I7 0 2002 -1 -1 -1 -1 -1 -1 -1 -1 25844 25608 85 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7780118309660496456 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8563276039202285702 1 Голос видео 1 2014-03-23T05:23:15 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/triket/4746734017811fc051dd0e57 http://yandex.ru;yandex.ru/?area=ShowForum.ru/look/event cdo.e1.ru.plays олины.рф 0 0 [] [] [] [] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 -1 0 0 0 1460 549 574 2014-03-24T05:30:03 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 215720129 0 0 0 0 0 5 2014-03-23T03:47:38 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 5 wi a5b3 0 -1 0 0 289 1 0 0 94 20 -1 -1 -1 488 414 -1 0 0 07d21f 0 [] 0 18236259283221887320 16190469452247787632 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 9 0 -8599500047480121116 1 Брюки с TurboBit.net - фильмы онлайн на 2014 - Финансий 1 2014-03-23T16:22:36 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://hurpass.com/ogrevolyni http://foto-395077193b7240786&lr=11110436606 photostrana cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [9,121] [] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:46:24 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 42215400 0 0 0 1 0 E 2014-03-23T23:46:39 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 18315239292404426864 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 1 -8602651835226449867 1 7 самая мамки для ваше может по суперма Страница 719 тыс. ответов 1 2014-03-23T12:16:45 2014-03-23 1785 2088542490 921202579dbab4e58eddb04f693854b3 57 9499572962667875395 1 56 122 http://911114788/38/celebekgaleri.hurriyet http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/downloadTo=&orderby=RelAmountry_id=22&ved=0CAUQjB0&url=http://money?rand_cruitstart=444305.952058.13951639370f280c133ad16ce2c79e7cab93c5a23X3 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 11 6ee9 1 1 0 0 2366248 3 3 схема сумерки 5-6 летовой устанния 0 0 1627 708 157 2014-03-23T22:11:02 5 1 24018 0 utf-8 401 0 0 0 8929720244896745512 460839648 0 0 0 0 0 E 2014-03-23T22:21:38 0 0 0 0 0 [] 114378192 d7364e6a7ddcbcf6dcccca7bd6b2807a -1 -1 -1 nD a5b3 0 -1 0 0 626 189 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6784513853171268256 7631625543297538199 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8607066140185696713 1 Отслеживаны - №1065 - ekşi 1 2014-03-23T16:28:07 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/weatherson.html#78efe9869.shtml?wmid=143&srcid=UxB0pAAA6I9d0CWaWE%3DfrSMw http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T06:03:11 0 0 0 0 utf-8 401 0 0 0 0 813102074 0 0 0 0 0 E 2014-03-24T08:04:40 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 15 110 78 102 0 0 2060 365 -1 -1 -1 2397 1361 -1 0 0 07d21f 0 [1555356] 0 6331778101536488133 11618756116771170416 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 8 0 -8611399304547034181 0 Современа - Футбольших 1 2014-03-23T16:22:51 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1188c56ff8058343682.1_29_et._30108879a9aa61ea73752719 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vdmlyLnJ1Lw%3D%26engine_volume2=40&sid=6&ved games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:23 0 0 0 0 utf-8 401 0 0 0 0 356217277 0 0 0 0 0 E 2014-03-24T07:51:41 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11767694963832011096 2937255180427836822 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 111 0 -8664317093542350977 1 Афиша - купить готовой менение сиUçuşlardo, Yazlık Burçlar - astopol.info - Classion AAAQ (250Hp) CVT (140 тыс. км., смотр объявлению в автобусов - Екатегория России, клиент 1 2014-03-23T20:34:25 2014-03-23 32821467 1157253311 482b48fd303d64bba86ec7965c6ed6dd67 207 2460387875241374418 0 56 4 http://rabota.ru/show%2F2014 http://read.php?act=8&ved=0CC85ViwdY8L4WG9pLWlKS0RJU3AeAxJuh wwww2.sports m.auto.yandex 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3815009 -1 0 0 0 1136 593 117 2014-03-23T06:05:10 6 0 39566 0 utf-8 330 0 0 0 7635035874316412186 1000586617 0 0 0 0 0 5 2014-03-24T14:17:28 39 1 2 70 51 [10,42,1,22,3,112,3773,103,5,51,15,4,3212,47] 1549383847 ed3b609e3397b12c56cca4469c881c38 17234 -1 7 wr a5b3 0 -1 0 0 179 57 0 0 212 103 -1 -1 -1 613 -1 -1 0 0 07d21f 0 [] 0 3832538266798636143 15412096237897967728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 22 0 -8665395302091187491 0 Современа - Футбольших 1 2014-03-23T16:22:43 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=106/cid=191&rpt=simages%2Fthumb%2Fimage&lr=157 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20v&com_id=0&body_type=0&clid=1980&s_tg=&tariniz games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:12 0 0 0 0 utf-8 401 0 0 0 0 588577730 0 0 0 0 0 E 2014-03-24T07:51:28 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 1560098688466543352 11767694963289811056 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 121 0 -8673158852214232182 1 Голая Нагородской областической 1 2014-03-23T03:51:08 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/load.ru/#hs http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T03:26:28 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 270908049 0 0 0 0 0 E 2014-03-23T02:24:56 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 55446 -1 3 wi a5b3 0 -1 31 93 484 10 0 0 546 176 -1 -1 -1 1135 -1 -1 0 0 07d21f 0 [] 0 4313994922287858055 18236259282679687280 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -8725981425210147796 0 Аудио ЭХО Москве, ножений вопросова нашлось 397 1 2014-03-23T02:41:19 2014-03-23 15665476 1648690379 04eb9547b256147702f34bd2464beaf0 196 1425686864647023563 1 56 3 http://ivona.ru/top.rbc.ru/moscow=&q= Erection27=&toName=false&morpholog/vakansition&ref=city/daily http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/news.rambler.ru/search/ex09KBBcpWRDlaYTZYNEJHZDlOb3VHdjNWRXg5UFBpV2pFQ1qUQ&b64e=2&output=uds_app_id=19855479&view?ad=44&s_yers.don.ru/context=Sex&geo=0&s_tg=&offset=2.\\tПоповорого план кубе&clid=20639&keyno=0&l10n=ru&mc=1.58496250001978934&Lt=6&refresh=1&has_photofunia.ru/#!/video/search;web;;%2Fweb%2Fitem%2Ftitle,pos,p1,source=web&text=текстра фото 2014/view/125329d08a194e758644-500h_bWOg&bvm=bv.62922401,d.bGE&cad=rjt market.ru.platnye-igrydlja tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [] [] [28,267] 1622 1583 29 8 0 0. 0 0 10 73d7 1 1 0 0 1808122 3 3 интернет магазин финес и физическу о приколеснока, валерия 0 0 2011 768 157 2014-03-23T04:34:29 5 1 19832 0 utf-8 330 0 0 0 6243379005186084238 8777098 0 0 0 0 0 E 2014-03-23T21:50:32 16 2 3 854 0 [2193,82,265,3275,347,14,72,925,100,3,80] 1245455042 891bf70623c57bfdd5e9d09d616390eb -1 -1 -1 nD Tp 0 -1 0 72 206 84 12371 0 -1 0 -1 -1 -1 18 -1 -1 0 0 07d21f 0 [] 0 2560641399298933162 12328847313983957104 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8746280778383767298 1 Поздравоохранул 1 2014-03-23T13:55:17 2014-03-23 1785 3609570631 e9fd65b19c652c4ce85594f2fcd0c7db 196 593475812248875581 1 227 105 http://911114783342507357617c9eb279bb4f2642 http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/video/index.ua/pole-2167ba0f7c355305-nam-v-pid2-290622263757&m=3,5&Volume[1]=&extras[24]=0&eae=0&clid=22876.26.2.0.html&docid=onair.com/galeride/Kariy_el/zafiruet-onlajn_tr_5Gn0cFj_bANt_S8ROdvfrOg6pCgU5XY2P3MtHkZQNDILDhspeAAhWjJJgkFq5HXGrEC5GmjXQ5bGV3TTVeklJakNwelhKc0hDcjFTVTFWb180ff0d9&url=http:%2F%2Foktavito1296_3346d0f0;dcid=ya_wireleva.ru/yandex.ru/?action=page-67600&groups[]=17704&groups[]=13&msid=91cc2424241d2cd8128&lr=97541¬custom_key=&haggle_AO_Mondelevizit-dvigator.tv/tnt/?region=cadf2922401,d.bGE&cad=rjt coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 523 617 57 0 0 0 0 44 73d7 1 1 1 0 3684816 3 3 алекс старт 20кг купить десан автобазаречь создать 0 0 592 310 157 2014-03-23T22:38:38 0 0 0 0 utf-8 401 0 0 0 8382449486543033592 279598542 0 0 0 0 0 E 2014-03-23T03:43:12 16 1 3 119 4 [76,1] 3886548777 f69565e66e18aeaf2059a8b6495c723613 -1 -1 -1 nD a5b3 0 -1 0 28 22 9 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 4244945004353045279 5931335146677711795 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8931522039698213139 1 В пробегом - катеристочник 1 2014-03-23T16:28:08 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:56:58 0 0 0 0 utf-8 401 0 0 0 0 126947969 0 0 0 0 0 E 2014-03-24T08:00:54 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 22 23 62 61 0 0 996 0 -1 -1 -1 1043 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 240 0 -8933906015862790098 1 Афиша - купить готовой менение сиUçuşlardo, Yazlık Burçlar - astopol.info - Classion AAAQ (250Hp) CVT (140 Hp) 2006 купить компании Украина вытащил о Москвы :: Голосуток в 1 2014-03-23T20:34:36 2014-03-23 32821467 1157253311 482b48fd303d64bba86ec7965c6ed6dd67 207 2460387875241374418 0 56 4 http://rabota.ru/new.mcko http://read.ru/yandsearch wwww2.sports m.auto.yandex 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3815009 -1 0 0 0 1136 593 117 2014-03-23T06:05:24 6 0 39566 0 utf-8 330 0 0 0 7635035874316412186 28170112 0 0 0 0 0 5 2014-03-24T14:17:36 39 1 2 70 51 [10,42,1,22,3,112,3773,103,5,51,15,4,3212,47] 1549383847 ed3b609e3397b12c56cca4469c881c38 17234 -1 7 wr a5b3 0 -1 0 0 106 58 0 0 92 34 -1 -1 -1 148 -1 -1 0 0 07d21f 0 [] 0 15412096238440167768 13753603333183694960 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 19 0 -8939870126547196026 1 Комная прессан Партира, акциях — фанфики к экзаментально и части Донецка | РБК daily Digital acro 1 2014-03-23T16:02:25 2014-03-23 1785 2922543887 1c01b17e6ee3c1a01828ecd318a6b581 59 8723854704339558313 1 56 2 http://9111147842,0.01178.1394999-prepovedeki http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net%2F1b.jpg","photos&p=3&page=18413930_handguides[]=2&sign=cf8f01c12489e0a7&uuinfo/view/93190876323017&l=sem&sign=213&nore[2]=0&search?p=законы финальных треуголоваров владимирова туалеты 10 дневники андроид&lr=20525944&lr=1637&keyno=0&room=0&page=576&m[]=100&search/retsiatoust/avtobank.ru/#!/kemeye-karechenkoi-denegro/brando.ua%2F17%2F204a%2F&ei=0-ochekiev.ua/opinion%26wheelpUTdBQUhtbTZhSzItYVZOOUjhfNF9vZDdGVzLzAwMzA3L3BwLzQxNjY2ZHpnY2VWxiCHsuJClHPnYMQVoxbSVTUFJTElsakI4WFp6NHNHJRDRhOU1LR3BuSmJYYj9tOUmxUOXMtVlN1TjNtVGc1c032a1b3672 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1297 1010 57 8 0 0. 0 0 18 73d7 1 1 0 0 2992777 3 3 детский номер электрообная игры скорота фото кало 0 0 1366 391 157 2014-03-23T00:56:51 0 0 0 0 utf-8 330 0 0 0 4847654860042290847 969371126 0 0 0 0 0 E 2014-03-23T02:42:26 55 1 3 12257 0 [10,9,112,68,365,76,260,22,1,2817,3,3540] 2380200681 0735dfb043075c30693475487bf24a49b7 -1 -1 -1 nD a5b3 0 -1 1 55 84 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 5849787649459211608 17662636599668303984 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -8964700706069591280 1 Салата 10. Цензор видео, онлайн (все забилей с пройти в Мельного 1 2014-03-23T04:19:17 2014-03-23 22447063 1796001934 3ae86dda5556b7f4ef38aa8077f6825a 15887 1644674872295047497 0 3 4 http://images.yandex.ru/neo2/#inbox/&dates=0&run[1 http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 2 [353] [] [] [28] 1587 555 57 10 13 0.77 0 0 25 73d7 1 1 0 0 3996694 3 2 0 0 1713 555 117 2014-03-23T19:14:50 0 0 0 0 utf-8 401 0 0 0 0 939725270 0 0 0 0 0 E 2014-03-23T02:46:14 0 0 0 0 0 [] 1907295579 dc652869f8d8eff9ed5b5daa5374b163 5038 -1 3 wi a5b3 0 -1 305 0 452 30 0 0 4029 164 -1 -1 -1 5023 6719 -1 0 0 07d21f 0 [] 0 4313994922287858055 9730402382055663728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -9065033574850035452 1 Серия 9 - 2013. Дети@Mail.Ru — социалист 1 2014-03-23T16:20:14 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://yandsearch?lr=35&rpt=simage&lr=66&text=розаврилова http://google.com/games.mail.ru ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 3 2 0 0 1994 561 117 2014-03-23T06:43:38 0 0 0 0 utf-8 401 0 0 0 9007862211105217806 42215400 0 0 0 0 0 E 2014-03-23T23:44:06 16 1 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 47 40 590 134 0 0 2110 -1 -1 -1 -1 -1 3460 -1 0 0 07d21f 0 [] 0 1930741556500475038 9207967286803898539 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -9186659792231785281 1 Настройствоваться частных (б/у) автор о продукции; развития детьми - Грузов 1 2014-03-23T09:48:08 2014-03-23 1785 1341248658 6366e773993d35514d6b846f79b34292 183 626923241082458450 1 56 4 http://9111147832977565.html?cat=420801;label=perioda http://go.mail.yandsearch coccoc.com yandex.ru 0 0 [3,5,6,15,57,91,76,175,353,403,349,3121,11503,11630] [119,3418,2455,3036] [18] [155] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3500889 3 2 0 0 1997 548 117 2014-03-23T17:20:17 0 0 0 0 utf-8 330 0 0 0 8416052423457712746 215289560 0 0 0 0 0 E 2014-03-23T23:14:07 16 2 2 13621 14 [14,72,80,925,370,109,7,285,3274,101,6,66] 1160820115 524ee8575739a6149a641e6f4fbc6f7b -1 -1 -1 nD a5b3 0 -1 9 45 109 42 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8578740285396261239 4548538545130905100 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 - --- !sql13 -- -4632802378222380466 1 null Acoper «labilir mitsubishi в Липецке на Batak 1 2014-03-23T16:23:07 2014-03-23 27528801 2723656327 7fc6e55c74e3029834ae16bf67d8cb9f4f 15887 2224216775939367616 0 56 4 http://rutube.ru/patianu http://love/v012/04 footki.yandex.ru m.fotoalmeta 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3238011 -1 0 0 0 1136 555 117 2014-03-23T06:04:09 5 1 19832 0 utf-8 401 0 0 0 0 1018986580 0 0 0 0 0 5 2014-03-24T08:02:03 16 2 2 0 0 [] 2529497135 30d877a66ec8945d1e4d056e51e3de35 18372 -1 2 wi a5b3 0 -1 4333 1332 747 2472 0 0 7395 285 -1 -1 -1 16115 -1 -1 0 0 07d21f 0 [] 0 307141980878902893 338497349483004835 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 47 0 -4716566754160586468 1 Тимошенковске на F1news: Новгородажа легководителей и ее европы - Странспорт 1 2014-03-23T10:52:43 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.139517&quickpantinoyleri http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 943 296 2014-03-23T22:23:20 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 278406132 0 0 0 0 0 E 2014-03-23T20:23:36 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 0 0 1 2 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 7982945165681148628 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -4755557211339260600 1 PwC 1 2014-03-23T16:25:12 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/index?appkey=&m[]=6&frommail.yandsearch?text=никол в про и невиннисаж http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T05:54:31 0 0 0 0 utf-8 401 0 0 0 0 346099590 0 0 0 0 0 E 2014-03-24T08:01:40 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 59 78 0 0 2251 0 -1 -1 -1 3362 798 -1 0 0 07d21f 0 [1555356] 0 6331778101536488133 6157439411447496521 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -4770046483886760822 1 Mercedes-Benz e-клас. Можно купить 1 2014-03-23T16:26:50 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/weatherstyle/29/127737064&domain=hurriyet.com/volgogram/22/10/Default-tistings=535674/atturkiye http://mynet.com/mate-guzellileridetay/80508/bio/video?q=породажа хабаров с технологических mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T06:02:18 0 0 0 0 utf-8 401 0 0 0 0 782597291 0 0 0 0 0 E 2014-03-24T08:03:20 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 96 115 252 0 0 6165 306 -1 -1 -1 8173 1287 -1 0 0 07d21f 0 [1555356] 0 8744694472066974558 13107909194127435888 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -4831817364509130205 0 Сериалы 1 2014-03-23T20:34:29 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 http://yandsearch.php?id=209516817950 tranamaschools top.rusfisha.msu 0 0 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 -1 0 0 0 1 1 117 2014-03-23T06:05:12 0 0 0 0 utf-8 401 0 0 0 0 6030782 0 0 0 0 0 E 2014-03-24T14:17:35 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 -1 0 0 3018 0 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11033243704253757784 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -4835752945340096735 0 Продать, предложений — AVITO.ru. Часть бесплатно онлайн Фильма «T+2» (286): 1 2014-03-23T12:16:06 2014-03-23 1785 44808656 0442e62a5c5cb134d699fbae72bc9c588b 7 454237030118460538 1 3 3 http://91111478334250732/?page=20&i=1&parenk=&changed http://yandsearch;web;;%2Fwwwww.silverdi-kading/site_id=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJpvc9LIo8yqpc1a7i7QPYfRisPDxcOAMMjcyNjQ3MTIxfcSMTQ0Y2U4OWQ5WWVFUnFSWhVTUNjVmN2N2 coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1297 555 29 8 0 0. 0 0 17 73d7 1 1 0 0 1384917 3 2 0 0 1366 1019 209 2014-03-23T20:26:57 3 0 58982 0 utf-8 401 0 0 0 5636140508787295899 929361662 0 0 0 0 0 E 2014-03-23T22:24:51 55 2 2 675 0 [4,5,80,82,2193,285,265,76,14,10,112,3,9] 46871203 be72ce3c669bd45ee99b8e038d989526 -1 -1 -1 nD Tp 0 -1 0 0 19 11 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10605622915436919128 11933878103501891696 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -4837660920166529023 1 Сбербанк ОнЛ@йн 1 2014-03-23T13:27:36 2014-03-23 1785 280750947 99ebef8641c09581865c6e5c72b33f05895a 19973 322556318799166472 1 56 2 http://9111147842,0.01178.13950555&lr=66&img_url http://kelebekgaleri.net/besplayer&join.ua/rezerval=1&bodystyle.com/8-marter97.org/remont/innopoisk.ru/news.mail.ru/video/search.xml?categoryId=5000&year][max]=1002-fh-467-pd-1&rpt=simages.yandex/uchita.ru/view/133-ukrainy-duoi tronika.html?l=prod.dp.ua/?page/61307-fh-467-pd-1&p=17&uinfo.aspx?wt_rp=27&hid=91651068.html&lr=213&text=search?lr=2177467203159_5500000191&text=бекление&filmId=P90GJOSRzJSNWM%3DfnSZA%3DfeSfgSMQ%3DfoSfpSYmJTNFRFJzVjdkRNQ2VWRoUFNuTDJPZnJ0U3ZnhaMlJ1bEhoVTE2WFJRTzJ1bUl9HP3J5ftXDKDyocZuDO7C4iTUDb8chOvsX2R1YmU9ecE4j44QTjhNRDRpRE40ZFdElB&b64e=2&sign=89846355c1c50b406146965/36733f0427100&sfrom=592&m[]=5431ecedes_businiz-view957448172/?from=yandex.ru;yandex.ru/a-sezon coccoc.com sprashivai.ru 0 0 [5,353,3121,11503,11630] [119,3418,2455,3036] [18] [28] 1339 555 57 8 0 0. 0 0 29 73d7 1 1 0 0 1412357 3 3 ходячие раскрасны 0 0 1997 891 209 2014-03-23T20:32:22 0 0 0 0 utf-8 330 0 0 0 8168139702751979718 408832541 0 0 0 0 0 E 2014-03-23T23:57:58 16 2 2 2 0 [9,3274,1,14] 421494715 f20bb2faf420abe95559d6bcfe354d05 -1 -1 -1 nD a5b3 0 -1 0 35 58 56 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 18126425332249278808 14950347525130479728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -4850926408483995792 0 Современа - Футбольших 1 2014-03-23T16:23:19 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1_844_128338&text=ниссат б3 2.0 скачать http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vcG5ld3MtUUov2KB1GZjamJIeW1nQ3NGMkVMNEZBa3hpZ2VzLzI3L25ld3MvMjgw games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:46:01 0 0 0 0 utf-8 401 0 0 0 0 583663939 0 0 0 0 0 E 2014-03-24T07:52:16 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11875920750199978328 4452926215554207674 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 107 0 -4926263996196211822 1 В пробегом - катеристочник 1 2014-03-23T16:21:04 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T05:53:35 0 0 0 0 utf-8 401 0 0 0 0 1056965251 0 0 0 0 0 E 2014-03-24T07:57:58 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 21 49 40 104 0 526 0 -1 -1 -1 1986 1645 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -5017198962525199154 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:57:36 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:36 0 0 0 0 utf-8 401 0 0 0 0 658682892 0 0 0 0 0 c 2014-03-24T07:38:20 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 17131 -1 1 wi a5b3 0 -1 0 0 1 3 0 0 241 0 -1 -1 -1 410 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -5105151702377877258 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:12:37 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:43:14 0 0 0 0 utf-8 330 0 0 0 0 419812360 0 0 0 0 0 5 2014-03-23T11:13:03 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 2 nD Tp 0 -1 0 0 287 327 12 0 529 1480 -1 -1 -1 1501 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 750 0 -5138975406131988633 1 Сбербанк ОнЛ@йн 1 2014-03-23T13:28:24 2014-03-23 1785 280750947 99ebef8641c09581865c6e5c72b33f05895a 19973 322556318799166472 1 56 2 http://9111147842,0.01178.13950555&lr=66&img_url http://kelebekgaleri.net/besplayer&join.ua/rezerval=1&bodystyle.com/8-marter97.org/remont/innopoisk.ru/news.mail.ru/video/search.xml?categoryId=5000&year][max]=1002-fh-467-pd-1&rpt=simages.yandex/uchita.ru/view/133-ukrainy-duoi tronika.html?l=prod.dp.ua/?page/61307-fh-467-pd-1&p=17&uinfo.aspx?wt_rp=27&hid=91651068.html&lr=213&text=search?lr=2177467203159_5500000191&text=бекление&filmId=P90GJOSRzJSNWM%3DfnSZA%3DfeSfgSMQ%3DfoSfpSYmJTNFRFJzVjdkRNQ2VWRoUFNuTDJPZnJ0U3ZnhaMlJ1bEhoVTE2WFJRTzJ1bUl9HP3J5ftXDKDyocZuDO7C4iTUDb8chOvsX2R1YmU9ecE4j44QTjhNRDRpRE40ZFdElB&b64e=2&sign=89846355c1c50b406146965/36733f0427100&sfrom=592&m[]=5431ecedes_businiz-view957498&answer=&channe_2010_hand.ru/chase/aleksey510376 coccoc.com sprashivai.ru 0 0 [5,353,3121,11503,11630] [119,3418,2455,3036] [18] [28] 1339 555 57 8 0 0. 0 0 29 73d7 1 1 0 0 1412357 3 3 ходячие раскрасны 0 0 1997 891 209 2014-03-23T20:33:18 0 0 0 0 utf-8 330 0 0 0 8168139702751979718 668154955 0 0 0 0 0 E 2014-03-23T23:58:33 16 1 2 2 0 [9,3274,1,14] 421494715 f20bb2faf420abe95559d6bcfe354d05 -1 -1 -1 nD a5b3 0 -1 0 0 14 2 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10100085969557141848 14950347525130479728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5291067970604774260 1 Максай игру роботы, купить в интернет-магазин спецпредства воорешебниках - вакансии лет — Сеть легководставито 1 2014-03-23T05:07:16 2014-03-23 15665476 1087886866 269c5c27e4946a667f78888c9f75b35c6e6e 49701 334328984627246488 1 56 5 http://mail=1&search?text=гугл перевод старый сайт с шоп&strirodam_spanie-podkRPaXptc3hrVndyR2JaRFUxWTJ4aFVG http://hood_search&event_id%3D84840.2402105-50&uuid=&status=0;orderevery-club.me/berkusurulu-bolum/2522&language=2014&page2/?go market.yandex.ru.msn.com pegasha-pogoda 0 0 [353,3121,11503,11630] [] [] [28] 1174 809 29 8 0 0.77 0 0 15 73d7 1 1 0 0 3560775 3 4 погода на рассаж 7 0 0 1962 676 157 2014-03-23T06:41:45 0 0 0 0 utf-8 401 0 0 0 0 419939021 0 0 0 0 0 E 2014-03-23T00:07:02 55 2 3 32 103 [2193,265,82,6,95,288,14,100,3275,72,3,76] 1477819126 1be50e4c7673a88c47cace3e47478733 -1 -1 -1 nD Tp 0 -1 0 0 3 30 61 0 25 -1 -1 -1 -1 -1 52 -1 0 0 07d21f 0 [] 0 8851578361410429238 10331158898916649072 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5305634367927610522 0 Современа - Футбольших 1 2014-03-23T16:23:48 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=19554212079d1b7cbb8a6a2.jpeg%3DfeSfgSYmI%3DfdSNTU2feSMzAw http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb21f3fc3aaa24dc63a21xZGVqSE1GcFQxUkJN&b64e=2 games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:46:25 0 0 0 0 utf-8 401 0 0 0 0 167969307 0 0 0 0 0 E 2014-03-24T07:52:45 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 3480781366221101891 1661165035492520768 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 127 0 -5309644206721379498 1 Письмоградской 1 2014-03-23T20:55:44 2014-03-23 27941559 1767624985 a7da21818799159fb389b18d4373596b 225 2721568806677236614 1 56 3 http://patia_spark ns7&search?clid=9582 http://hood_search&event=list/casing-cap tvmir.nu pegasha-pogoda 0 0 [353,3121,11503,11630] [5,7,73,400] [18] [28] 1174 809 29 8 0 0. 0 0 17 73d7 1 1 0 0 2179614 3 4 герб марта спицам терми 0 0 1731 391 117 2014-03-23T05:14:07 0 0 0 0 utf-8 401 0 0 0 0 777203605 0 0 0 0 0 E 2014-03-23T13:07:48 0 0 0 0 0 [] 1852761877 b016f1c7c4de86f1c384f8d63570a742 -1 -1 -1 nD Tp 0 -1 0 0 460 199 31 0 1530 0 -1 -1 -1 2846 -1 -1 0 0 07d21f 0 [] 0 15279646984699132248 8418678203875343898 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5420983836600446463 1 Продажа Toyota / Результиворк! 1 2014-03-23T03:35:14 2014-03-23 1785 1950378921 c2602efb1311636cf875df404f3d6529 51 3206084085559802010 1 56 4 http://9111147854&change_key=506d9e3dfbd268e6b6630e58&domain=hurriyet http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ukrayila-yenisafak.com%2Ftitle,pos,p0,source,web&text=кредиторожный краска длинны онлайн&uuid=0&count=135238478226d6ff8ed00dMU7UvmIoQmkzfPmqekNCSExDRVoxZEcxcGJsN3kxUUxhalleonary_to=2014 sony_price_usd[1]=&proigry-dlya-suka.ru/sankt-peterburgskaya_oblast_volume[2]=&color_id=0&year[1]=&private/en/sessuary?p=90&q=1395592f72162c019715ade9befae9562669,"first_name":"Альберт&clid=195545452&lr=109aa7f2a8bc3d75c95724332 coccoc.com tanks.co.il 0 0 [] [119,3418,2455,3036] [18] [] 1297 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 906571 3 3 торта самарекс по оптом из неполных года выхода скалы фотошоп 0 0 1366 593 262 2014-03-23T16:44:09 5 1 14470 0 utf-8 401 0 0 0 4808217662922694360 7065527 0 0 0 0 0 E 2014-03-23T16:14:51 0 0 0 0 0 [] 2137570165 46a233a0e33f11c4fee726c7303f44d5 -1 -1 -1 nD a5b3 0 -1 1 173 292 430 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8411534898364058195 7194548681272151755 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5520081962326584152 0 Ремком крае национны [AVC], VA, Любой в Москве крымская историгина домашними рукодекс: нашлось 3 млн ответов 1 2014-03-23T09:45:17 2014-03-23 1785 1242723321 2684553455bcb892472422768c7b4b2f 13437 3074483810024357617 1 56 2 http://9111147842,0.01178.13956069c61cc8Apdghvwm6ThW9Y4QOdYgU_GUIzigLM8W6g0QMWtNUXI&where=all&filter coccoc.com 0 0 [] [119,3418,2455,3036] [18] [] 1622 1583 57 10 6 0.70 0 0 14 RT 1 1 0 0 3039699 0 0 0 0 2011 726 296 2014-03-23T18:39:54 5 1 19832 0 utf-8 330 0 0 0 8207959345338106813 992900079 0 0 0 0 0 E 2014-03-23T20:03:28 22 0 3 13732 6 [6,219,1549,7,3003,501,999,18,1,36,25] 2143328713 fcb87cf6e44fbb0e1d8456015c663d3786 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4127982339363240333 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 96 0 -5559361293957300414 0 Женская библи (сериал "Фиксика 1 2014-03-23T18:17:33 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/forums.kuban.ru/volgogradmir http://yandex.net/legkovye-avtorii/118534731400928 yandex.ru tury.ru.msn 0 0 [85,350,1212,3849,8358] [106,323,1092,3676,13188] [18] [28] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 -1 0 0 0 592 503 117 2014-03-23T19:19:06 0 0 0 0 utf-8 401 0 0 0 0 647923622 0 0 0 0 0 E 2014-03-23T12:26:14 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 1166 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 884992008826344665 4724303053209794720 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -5619100536162139501 0 QIP.RU - Гайд Dota Construment Steel, Красности стильни госдеп США Джинс | Вакансии. Одежда деловек пау газета.Ru — смотреть лительность ерально и в Новости Украина, фото - View Film izle Çizmet Kay - VAZ, Gereketim Dinlementique – сайте AUTO.RIA - Фишер Успевай 1 2014-03-23T12:34:12 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://9111147854&change_key=506d9e3dfbd268e6b6630e58&domain http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/view_intext=как в москва режим умом заводоровать онлайн бесплатно в любимому языку 4 класс порно встрашный сайт длинновая папины мамедостовск-орловского ли реимущественны и холодилер шеференков&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XsltfjMh8My-4Mp2Kzonmlr9Oj-tkmzK1-_9gEp7c156bcf985&text=sony?p=11&cbv=r2014/03/19/bigparams/174780954.24549/2679&page10/?&p=36&uinfo/index.ru;yandex.ru/krasnodar.org&from=yandex.ru/viewly&kinda-raduga/arams.com/haber/10/155-02-20053.4678203611557414e2725-derby_day=2014&clid=11985&lan=ru&mc=4.804252639b7ba5981421beb&ui=uv3lovAaf3sLhCa43t3Qfk-sdelaya/sobaki/?page/dl/секс поручную под люди&lr=213&text=гарнир__Абсолютное подтяжки&lr=108444071-fiksiki-6-chere=all&filter_id=13451&lr=8&text=диета авито блей&lr=66&bpp=7&i=1&when=2009%26nord%3D%26extras[14]=0&extras[27]=29066462.0819084f155715641255491/dating/used/sale%26d%3D26%238242_259066068085843223.html?ia=M&lf=M&af=18&pw=1&target=search.phtml?pid=89§ion1[3]=&searchastnik.ru/my/hit_list/work.ua/clck/jsredirected=13859&text-albinoy-kutladikavkaz/makler.ru/aclk?sa=l&ai=C9QzwyeUQVCiFZB79rQYLQ1GPgN7Qi82fGpeBl0LLWFkQlRDAlQjUlRDAlQjklMjQ5dWFWOW9iM1I1WVhKdld5NDV1MOXXox_OxcFDY-uop-thersoy-tonkiimir_snovachale-secim.html#/battlemena.avito.ru/cars.auto.drom.ua/manage=eyAidW5yZWFsdHkvMjAxNDAzMTgiO3M6MjoiaWQiO3M6ODoiMjAxNzE0NDk0NzUva$84NzI2Mjk0NzttYAw&usg=AFQjCNGheBpruD0hte0FyaUqCNDk3DqQvygvalcenkin/offers.xml?category_id=199555453&banner_pf[HM_S106Wpb7rjrgxoAGGkuc2svdGhlYXRyemst=my.mos.ru/yandex.ru/companoritecomm coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 орхидея в мирандашом ценарисы гузеевой мото рисунок и простройщика фото с поздравлений телеканка фото 2014 год собласти ужгород 0 0 1389 884 157 2014-03-23T22:34:54 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 846684171 0 0 0 0 0 E 2014-03-23T22:48:13 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10287839608055806296 17253681420892301424 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 89 0 -5711297485370126517 1 Голос видео 1 2014-03-23T03:51:10 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/triket/4746734017811fc051dd0e57 http://yandex.ru;yandex.ru/?area=ShowForum.ru/look/event cdo.e1.ru.plays олины.рф 0 0 [] [] [] [] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 -1 0 0 0 1460 549 574 2014-03-24T03:26:32 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 585440942 0 0 0 0 0 5 2014-03-23T02:24:58 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 55446 -1 5 wi a5b3 0 -1 0 0 454 3 0 0 78 37 -1 -1 -1 386 486 -1 0 0 07d21f 0 [] 0 18236259283221887320 16190469452247787632 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -5733212747584636203 1 Как лучшие руках, часа в Красноярска обновлетворящих » Купить качесть по г. Москве на AVITO.ru (Недвижимость на AVITO.ru. Стулья салоны в Владимир 1 2014-03-23T13:20:49 2014-03-23 1785 2501928799 4a31e16945f0d802d9dc0705979c4672 239 4517116963127554624 1 79 4 http://9111147811.html?v=1393_0_251311310 http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 2081850 3 2 0 0 1997 593 441 2014-03-23T03:13:35 0 0 0 0 utf-8 330 0 0 0 7146369058166890546 793191128 0 0 0 0 0 E 2014-03-23T23:56:34 16 1 3 12333 0 [14,1,6,501,119,72,9,510,3540,925,22,11] 2976974125 a3831324980206d1415afea636cc7635 -1 -1 -1 nD a5b3 0 -1 92 125 121 201 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 12557946248999135344 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5772344969531870975 1 «Сбербайджане, вакансий и джемпер цены найдено в Яндекс.Работа для занятия числе за 450 000 рублей ПК | Купить Ladie | News.ru - Новости • Продаже к Eurospor-Fenerbahçe Speed: Resmi intine траханик, - Politikam alın mustafa Keyfiniti JX внедопуска | mp3, видео – частных фото эротив Украинская Компаний в Московским 1 2014-03-23T03:16:04 2014-03-23 1529 374391783 e3b7a9be7902b95c5cf957839f5c66920e71 154 1688129825915497537 1 56 3 http://realty.dmir.ru/yandex.kz/Tsvet-risunki http://apps/docview/Kvasi.html&ei=cL5HMPDC31TNXJ0MHZlWHNVctY2ozU0EzNUJVddxVjBmc1ld9IX1iEAbgEDgu kolesa.kz forum.print.ru.msn.com.travel 0 0 [2672,3170] [5,7,96,420,477] [] [28] 1846 849 29 8 0 0. 0 0 24 73d7 1 1 0 0 1072276 3 12 поезда молдованной 0 0 785 832 296 2014-03-23T00:08:04 0 0 0 0 utf-8 401 0 0 0 6192436962476724500 979256876 0 0 0 0 0 E 2014-03-23T20:34:42 0 0 0 0 0 [] 327000091 3fe367918369045361fae91e8d77b859 -1 -1 -1 nD Tp 0 -1 49 0 642 1 1637 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6855930052452742827 6759968051075183589 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5869363305929907643 1 GALAXY S4 milyon sonuç bulundu! | Независтная можешь не с коттеджи - Новости: чемпион» 1 2014-03-23T04:15:40 2014-03-23 15665476 3911193819 75af3860b05c745c5c5c27b30f6ab2ed546589 196 527703306321131114 1 56 4 http://lima-lgot.php?t=7684-3c1d3ca8bf948f9add4993509/6257644%2F04725054%2F661%2F52499 http://yandex.ru/yandsearch?lr=213&oprnd=9161312&search?text=уход мерседес аста сараты&lr=2&tag=studen-twittemplatyana-ne-pistoryid=84643938c0395/album_id=19720.com/fuatavni--4292-5-0-271-zazda_3_/?curre=0&shv=r20140316862,bs.1,d.bGE&cad=rjt images.yandex.ru.msn.com.ua yandex.ua 0 0 [353,3121,6479,8856,11503,13770,14847,11630] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 1769215 3 1 в россии отности 0 0 1789 602 157 2014-03-23T05:44:47 0 0 0 0 utf-8 401 0 0 0 6176299642330297673 798409806 0 0 0 0 0 E 2014-03-23T23:18:58 16 2 3 11502 7 [6,2,119,95,2193,5,272,109,3275,1,28,70,14] 2533100459 6bc04605521abb96d790e6be4375ac1f -1 -1 -1 nD a5b3 0 -1 1 55 213 0 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2391497380856937231 16719441483451263088 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -5872091609206767166 1 В пробегом - катеристочник 1 2014-03-23T16:17:06 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:48:18 0 0 0 0 utf-8 401 0 0 0 0 1035856160 0 0 0 0 0 E 2014-03-24T07:53:45 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 22 31 62 42 0 0 1327 0 -1 -1 -1 1381 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 252 0 -5894101013807685227 1 Kia Royal manga online, MediaPortaya barsu — стройки - Страну г. Модная фанфик пришли с животных ведущий Волк 2 млн ответов посмотреть 1 2014-03-23T11:16:53 2014-03-23 1785 2750008691 d27e873642fdcc142fd52f628470ba33 241 7406071195777269826 1 156 122 http://911114786987.html#/battle-en47.html?field/rd.dometrov http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/sitelink_id=0&year[2]=&extras[23]=0&search.xml?&model=false&showtopic.php?w=728&text=пдд 20140321%2F17-032-GWAY&text=личный связной&relocatid=207&q=samsung&q_page=0&client_id=200000943.aspx?DocID=240&sTo=0&l10n=ru&mc=4.12485408077879-PF=214240d26008706,2796&lr=35&win=20000&pmin=100&text=&etext=&search?lr=213&lr=960&t=o&sz=b&s_c=3159_37612330ea&keyno=0&l10n=ru&mc=5.4044324_40273&ga_vid=1&ved=0CIsBEIQcMCM&img_url=http://yandsearch?cl4url=aHR0cDovL21vc1ZOSW5AY9mSDAb-8Ep9GVzJ6R0xsNkVWRGT0D8ZTzZITkpPS2hHd058b3f7652/page=links,pos,p2,source,web&text=бэнтэн 10 коп 2014 иван недвижимость дому&rpt=nnetter-1-seriya-na-rabota.allbusinema coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 13 8w 1 1 0 0 2723983 3 3 натяжные моряк диски для короволос 0 0 1997 547 157 2014-03-23T21:33:37 0 0 0 0 utf-8 401 0 0 0 8546017116003926276 479894081 0 0 0 0 0 E 2014-03-23T21:51:09 22 2 2 0 0 [] 2771758996 1c0ec512cf36858257e85c74d2ac7654a2 -1 -1 -1 nD a5b3 0 -1 1 46 79 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 7713159565439621005 15167358802972043376 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -5920271052064151811 1 Письма сание 1 2014-03-23T16:27:56 2014-03-23 12461549 4071235651 fab2fa83e0438d1759eecbe52a5c5c2861 9580 532344165497878132 0 56 4 http://yandex.ru/c/11393489.0744 http://go.mail.ru&js=1&ref mail.yandex.ru yandex.ru 0 0 [353] [3666] [903] [28] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 2608898 3 2 0 0 1136 593 117 2014-03-23T05:51:52 3 0 51135 0 utf-8 401 0 0 0 4969423154180479309 317525830 0 0 0 0 0 E 2014-03-24T07:58:16 55 2 2 0 0 [] 2774799269 5998146c305c74c6c4bb7efb9b8586f333 -1 -1 -1 wi a5b3 0 -1 869 2 1010 390 0 0 578 0 -1 -1 -1 2739 2266 -1 0 0 07d21f 0 [] 0 4313994922287858055 3771131554017970634 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -5969822703110278505 1 Hyundai Accord в Новостелефоны 1 2014-03-23T05:23:01 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/for/tvorcher.taka.com http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T05:29:49 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 1073493585 0 0 0 0 0 E 2014-03-23T03:47:26 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 3 wi a5b3 0 -1 0 0 770 6 382 446 423 162 -1 -1 -1 1798 1877 -1 0 0 07d21f 0 [] 0 4313994922287858055 11715451638231263344 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 8 0 -6052478491822619321 1 Opel / Результик и быть суточных, серты на AllBoxing.net Email.Ru: Cemilie screed for movie Trans 1 2014-03-23T10:52:53 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://911114788/34113800297462973291/140/Default?prodau_dvukhkolaeva http://fotogrammyeli_i_ukraine-sims3packcpm.php?topic.php%3Ftarget coccoc.com m.mylove.mail.ru 0 0 [116,2586,3324,2253] [119,3418,2455,3036] [18] [28] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 -1 0 0 0 656 943 296 2014-03-23T22:23:33 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 434567472 0 0 0 0 0 E 2014-03-23T20:23:45 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 0 0 483 200 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 4982207522207701191 14629325741541222512 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -6133511635323783592 0 Доска без поступавная празделия положилище 1 2014-03-23T11:10:20 2014-03-23 1946 1936436644 2a53729da8c204bd5c6270cce24a479a4f 196 11626159163821345859 1 223 42 http://avito.ru/for/spravoslavl.upravili.ru yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 317 763 57 0 0 0 0 4 73d7 1 1 1 1 S820_ROW 647650 0 0 0 0 592 1639 157 2014-03-23T03:51:31 0 0 0 0 utf-8 401 0 0 0 0 729839554 0 0 0 0 0 E 2014-03-23T03:36:02 0 0 0 0 0 [] 1982208793 be2c1adef2c444e4237e7e2837889d34 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 2529151358065175971 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 0 0 -6144733558785969432 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:13:39 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:44:27 0 0 0 0 utf-8 330 0 0 0 0 358446447 0 0 0 0 0 5 2014-03-23T11:14:24 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 -1 0 0 11 105 28 0 353 23 -1 -1 -1 811 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 866 0 -6164438624456261208 0 Маринговых домашних услуги комнатную руками смотреть онлайн бесплатно 1 2014-03-23T21:12:12 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogo_baskalabs.html tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 0 412 567 296 2014-03-23T06:42:52 0 0 0 0 utf-8 330 0 0 0 0 529683494 0 0 0 0 0 E 2014-03-23T11:12:43 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 1 nD Tp 0 -1 47 0 251 0 7 0 168 12 -1 -1 -1 894 -1 -1 0 0 07d21f 0 [2170618] 0 15284527577228392792 2665691200259067948 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 793 0 -6209194776693341710 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:56:49 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 1 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:02 0 0 0 0 utf-8 401 0 0 0 0 968302720 0 0 0 0 0 E 2014-03-24T07:37:21 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 43140 -1 1 wi a5b3 0 -1 1 27 38 3 0 0 1791 0 -1 -1 -1 1897 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6223347640367363758 0 Задач на AUTO.RU: Поиск туры в Липницкой Радиоактивы -2015 год геев. Учены на SEfan.RU 17.03.2013 смотреть легковой закансии. Продажа легальном из ЕС бегства, запчастей. MuzMix.com - AVITO.ru — страница с России (56): Яндекс.Музыка для помогите популярности, динами: 4490210 1 2014-03-23T13:47:05 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://911114785-fw-112889455876.jpg","photostrator http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/clck/jsredirectadvert_month=2010&year=2002324.vkrug-orgtehnicspirit.ru/imgrefurl=www-898-ws40-10058/links,pos,p18,source,web&text=медованнал стс оформеропор"}]}&ref[0][paratovsk.ru%2F&lr=16&ct=adprev_fmts=728&uuid=&price_val1=&vol25/2/1/2/3/soru-kirdi/6567/47210%2F17mar1788&oprnd=49792&group-sedan/used/kampiyonlain-perednie-voennym_retro-cc.ru/yandex.ua/lenta.ru/catalogue/38939f0a656b977433296_33430-fw-1366&search;web;;%2Fweb%2Fitem%2Ftitle,pos,p16,source,web&text=недостановосибирск картинки&clid=157&oprnd=8782fef9e35b96627b4a237a8548.1&__utmx=-&__utmz=2111-iz-2755&lr=1036383bab44e22e6841483&sign=95&c=1&group_rul/cev-ustry=11882610&type=2&sign=cd73ec8b341b21cec73/23681912557914~EQ~sel~sel~x1561845/?i=1&w=экопольную информа между черезидения&uuid=&pricesmin=300&wp=1&searchText-seye-rovench-bank.ru/fore/1728&uuid=&subscribe_id=0&extras[2]=13658857.6753039669e18799961c\\\\%2F537475t2JFdkhSN1ZnNhdkx2M0NzE2VUl2WjNYanN6ak5ZVNMdzJSUWcwcDg5aUctaXhiZTVXTkU4V2VKd0toY0FrWkdiRkpLWEVCYo9vUg&usg=AFQjCNHCw82bldXYklOZnJyMDNVbVk&b64e=2&sign=43640d834a72f16b5872 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 орхидея головых отправоохожденных перечественниями 0 0 1389 884 157 2014-03-23T23:59:59 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 34846056 0 0 0 0 0 E 2014-03-23T00:15:22 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13223569481380149592 15473670750921746544 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 115 0 -6226406851137964111 0 Сериалы 1 2014-03-23T20:34:14 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 http://yandsearch.php?id=209516817950 tranamaschools top.rusfisha.msu 0 2 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 -1 0 0 0 1 1 117 2014-03-23T06:04:49 0 0 0 0 utf-8 401 0 0 0 0 122853544 0 0 0 0 0 E 2014-03-24T14:17:24 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 -1 0 0 935 0 0 0 285 0 -1 -1 -1 1165 -1 -1 0 0 07d21f 0 [] 0 11033243704253757784 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6229935597032769119 1 Серия 9 - 2013. Дети@Mail.Ru — социалист 1 2014-03-23T16:22:58 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://yandsearch?lr=35&rpt=simage&lr=66&text=розаврилова http://google.com/games.mail.ru ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 3 2 0 0 1994 561 117 2014-03-23T06:46:48 0 0 0 0 utf-8 401 0 0 0 9007862211105217806 845095662 0 0 0 0 0 E 2014-03-23T23:46:59 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 0 0 1 1 0 0 562 -1 -1 -1 -1 -1 303 -1 0 0 07d21f 0 [] 0 1930741556500475038 9207967286803898539 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -6269707891471564999 0 Путешества@Mail.Ru / Мои сообы сказка 1 2014-03-23T21:11:07 2014-03-23 16862487 1755644572 945af199eed21f3f249577b14f55534b 8363 309245865193991600 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carry_2013.html&forInitQuery=PER_OFFERS_SRCH_MAIN][]=&a1=&a2=30004989614 docviewer.yandex 0 0 [] [443,333,5760] [18] [] 0 0 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 2903448 0 0 0 0 0 0 117 2014-03-23T04:52:19 0 0 0 0 utf-8 330 0 0 0 9110769067793080479 287375675 0 0 0 0 0 E 2014-03-23T13:17:31 16 2 2 0 0 [] 1225767431 2672ee20d87b5c666121145dd76b3644cc 50936 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7914288637035474585 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 61 0 -6310212742328505656 0 Современа - Футбольших 1 2014-03-23T16:22:30 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1955498279&lr=2222_internatik.com.tr&callback http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20#.Uykh-uslug games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:01 0 0 0 0 utf-8 401 0 0 0 0 826263063 0 0 0 0 0 E 2014-03-24T07:51:18 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2316055356621577660 1824019458964194192 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 65 0 -6346147178174773361 1 Входящие 1 2014-03-22T00:02:43 2014-03-22 22846233 3605189702 f3c61a1ee8a8323de4c169db67afbc76 3 504457526759793798 0 3 4 http://mail.yandsearch?text=05&bt http://yandex.ru/pozhet webapteka.ru amalgama 0 0 [125,2919,3852,3467,3755,3849] [] [3] [267] 1846 849 29 8 0 0. 0 0 44 73d7 1 1 0 0 3734681 1 0 0 0 1460 894 157 2014-03-22T06:59:29 0 0 0 0 utf-8 330 0 0 0 8010019903753992422 436843801 0 0 0 0 0 E 2014-03-22T12:38:44 22 1 2 53 16 [4080,637,3085,2188,76,3,260] 2860016566 f4737a6a6d479c3e8f6c174e9422eab4 37171 -1 6 nD a5b3 0 -1 0 2 134 27 0 0 278 302 -1 -1 -1 630 1666 -1 0 0 07d21f 0 [] 0 7540904469640655877 3212143930063194622 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -6483163902225769757 0 журнал «Звезды и россия в машину себя 1 2014-03-23T09:21:53 2014-03-23 1785 611580825 62eb3a585c72f10e638f5217d090aba847 227 2784193748591992768 1 172 42 http://9111147833425073412912706553-pd-1.10 http://vk.com/bu-urun/touchshie_zhizn/zhiznesa/dlya_detes/?r=mail.net/ru/view/2330295776aefe844bHdCMTN3TVF2eGZma1_Kc_Qmun0fTvuCMGW_4PSAIvfP3WjneKtbBGjsbrsk coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 317 346 57 0 0 0 0 5 73d7 1 1 1 1 S820_ROW 1413987 3 15 автобусы для девочек рефератов красивые авито креплено 0 0 592 1623 117 2014-03-23T15:23:10 0 0 0 0 utf-8 401 0 0 0 5087914340569110724 906904915 0 0 0 0 0 E 2014-03-23T19:33:38 55 2 2 8643 14 [10,3,103,42,4,5,47,1,18,9,3993,2,25,14,44] 625776447 7e19356a37bff380c4da31eaa4f601d0 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15981034806496213336 10512538539660077168 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -6502069373992989721 0 Современа - Футбольших 1 2014-03-23T16:23:02 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1903-wh-793-1862278383&c3=ffffffffdf882f80144e5f158fdc http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vcG5ld3MuZ25lemRvcFZ3cXpwRzlKa2tXbWgtWA&b64e games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:37 0 0 0 0 utf-8 401 0 0 0 0 422809779 0 0 0 0 0 E 2014-03-24T07:51:49 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 3111198995473007829 15232020126171326576 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 110 0 -6569123909715121044 0 lady Gang Death project - Раздел - Спортаже и обманулись прямо сейчас бытовой армию: два на легальный 1 2014-03-23T12:37:38 2014-03-23 1785 2212435124 03754e335f9c8b7a73be034716d1e15c6e 2 4284437126406031019 1 56 2 http://9111147811.html?v=1393_0_2513-season_dekor-kupe/mather&page2 http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/sitelink_id=0&year][max]=&search?cl4url=http://lika-disc=0-0-ID8x90.html%26client=gulnogo.net/ru/view=list/?category/gazeta.ru/clck/jsredir?from=yandsearch%2F&ei=J505064327.am15.net/tags/мультики&client=bucking/quick,inline&limit=249&s=1&filter_float_num=600009.html?period=-30&m2=0&output_format=1&cnt=29.0.1878821db3ceb99b664efa093d256e38a1099118%2FbGlua3MmdGltZnBMVE5aSlRfamJlcERHZkFRtUG1VYXNwdEtoUm9IR2ROT3VvSF0WWKAM9MzhkNWJzDQQJTvVdhP0kzZzkzbWt2An5h6DAzjnuV3TTVmaVNmd05QSjF0YVd6MmF4NlpWZE1RcVBiUEtKdlRwUnctTlcydGQyYXo3Uzd05xX1ZgKHRonANzKBQCAB8Oc0iyQBwM&num=224&bid=&filter_float_pre,sitesek/hird-g1467/comment=139504/offererbank-folder:,atta=UlNrNmk5WktYejR0eWJFYk1LdmtxZ2lUR2d3lfbW5iaEx2ZGZhYW1zL2JpZ21pci5uZXQvbmVkJmfX8xIAEsQVgWCVtXe0B_d2ACemhNAhsaeVRof3lzXUlfR0pxCDBeR3t8EQMDQidTAVMiAwQWT3hCVklnY2pWQXImPWF-J28HfURDUyVmZvUHRqVVZoMW5LT1OIej7CRPQ coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1846 1640 57 8 0 0. 0 0 14 RT 1 1 0 0 2722972 3 3 зоофилипуты край ли ферб семьи рисунок найтли криста каранд песню амуравности акадеты на 20.03.2014г. уфе 0 0 1389 884 157 2014-03-23T22:38:44 0 0 0 0 utf-8 401 0 0 0 8071604882198479759 26134642 0 0 0 0 0 E 2014-03-23T22:50:56 16 2 1 734 0 [3,1,3993,66,2,14,81,2193,51,6,4,42,22,9] 2773423373 040fd94a35ad354e5c6eccd1e32503cebf -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6408603430287046814 4860176597772369010 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 70 0 -6617135007550525549 0 О вветов) для ALCATE&sortf=DT_LAST_PUBLICATE&sort). Регистрации, купить Volvo Sexy 1 2014-03-23T16:14:07 2014-03-23 8221043 1279830590 92b26a315c628ed65b05f41c7c483306e1 225 2861635209668344684 0 82 105 http://yandex.ru/neo2/#messages.yandex.ru/users.html#cmnt http://yandex.ru/cat.php?name":"Соломенять соникс-3м анал на украдуга.рф/idea.com/galler/turne.com.tr/write-avtozvuchatsky-guitar-3SXGLQ854SXQ?sira=7 ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 968 1640 57 0 0 0 0 26 73d7 1 1 1 1 LG Optimus 2179614 3 2 how to copystalk 230 км 0 0 592 1280 209 2014-03-23T06:40:55 0 0 0 0 utf-8 401 0 0 0 0 558315821 0 0 0 0 0 E 2014-03-23T23:37:35 16 1 3 4 0 [72,14,925,9] 1095027162 3d78d8dc2096c7ece3b38cca6d7a6b83 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 7834825270834824731 12435296265939118192 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -6685707648291291480 0 Бесплатные жилья и в руковичах 1 2014-03-23T21:20:28 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carfiles/2014/9119&redirektor_view/27409,221&msid http://yandex.ru/clck/jsredir?key=RUR&body=9&page=0&ad_info=ww-1038&blog/list/4.html#/battle/ffffa7eb48b17cc6d28e63\\\\%2Fvideo19838/pWTQtamxDUVlLajl6b0gymebel-pogoda.kz/p/bo4568&form=RurPaymenta.ru/felication_oblast/nice_usd[2]=&transmissional][to]=&drive_serviceId=8489_1366&tsort_offers&group=&is_app_users/skor.com/haberler.ru/volgogradskaya/state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJpNjfFDg3rinFADOKTQ_mAo56EGz3QA-GQV7Yv3dqVNrZRmuDjKoihTN1qGoWb9yiKeN3iZGLV8lxIdiTVh0R0hleFNaZWRXWmQyQzJFdnBGajQzUEdNSG96aGlkYWVtNks0UzZldmNpN21VEbe5WJTDK0m0tA729 docviewer.yandex news.yandex.ru 0 0 [] [443,333,5760] [18] [] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 1 0 0 0 430 384 117 2014-03-23T05:07:54 0 0 0 0 utf-8 330 0 0 0 5177399025069094312 567587409 0 0 0 0 0 E 2014-03-23T13:28:24 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13822395940870043992 4149647501617670018 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 87 0 -6699645891496674711 1 В пробегом - катеристочник 1 2014-03-23T16:21:18 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:51:15 0 0 0 0 utf-8 401 0 0 0 0 563587185 0 0 0 0 0 E 2014-03-24T07:56:38 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 16 22 83 63 0 0 1132 0 -1 -1 -1 1706 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 178 0 -6711080737389998250 0 321.Bölüm Son Dakika Haber7 - Обаме | 24Daily Digital DJ Mix#page 2) - Your 1.5.2 бель 3344 года в 1 2014-03-23T21:14:44 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp http://women.autoua.net/plug=180432 tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:45:32 0 0 0 0 utf-8 330 0 0 0 0 606583569 0 0 0 0 0 5 2014-03-23T11:15:37 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 -1 0 0 20 84 26 0 471 9 -1 -1 -1 908 -1 -1 0 0 07d21f 0 [2170618,2429698,3131362] 0 4337786887194029924 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 813 0 -6723139378573305403 0 Современа - Футбольших 1 2014-03-23T16:22:12 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite.html#/battle/ffffff-healty.ru/producers http://mynet.com/porno/page=1&search;web;;%2Fweb%2Fitem%2Ftitle,pos,p0,source,web games.mail.yandex naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:44:30 0 0 0 0 utf-8 401 0 0 0 0 230589762 0 0 0 0 0 E 2014-03-24T07:50:54 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 16147094646787160408 3267175271180936349 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 92 0 -6827875552437106781 1 Дневники | Классной потенцессону озерог. | Европагателе империи 1 2014-03-23T07:31:48 2014-03-23 1785 1197807518 855e0f20066e5f8b529935eca29dda63 4 9117921353016804935 1 56 4 http://911114783/7759&img_url=http://hurpass.com/gazetes http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2410771 3 2 0 0 1366 482 157 2014-03-23T17:00:10 5 1 19832 0 utf-8 401 0 0 0 8959031936800923576 283438122 0 0 0 0 0 E 2014-03-23T19:43:32 0 0 0 0 0 [] 1547029586 c60bbae8a114c907c02e79b3a5e7adbd -1 -1 -1 nD a5b3 0 -1 11 9 12 27 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 10632465148382210160 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -6835310931891846974 1 CAT-форум Первое поедет путевки женщин спально и развлекатеринбурs 1 2014-03-23T15:49:32 2014-03-23 1785 1771120080 7d75e5ce54e6243e703c6d896feff233 196 316119400541676494 1 56 4 http://911114786987.html?html%3Fid%2Fm.hurriyer http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 938654 3 2 0 0 1997 519 157 2014-03-23T01:46:50 0 0 0 0 utf-8 401 0 0 0 8615910193726028779 622662043 0 0 0 0 0 E 2014-03-23T05:01:22 22 2 2 4 0 [] 1272029586 a838ad35997d7a263f34a03154baa8d7 -1 -1 -1 nD a5b3 0 -1 4 3 23 100 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 5960367464715143360 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -6842827632255179584 0 1 2014-03-23T21:11:17 2014-03-23 16862487 1755644572 945af199eed21f3f249577b14f55534b 8363 309245865193991600 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carry_2013.html&forInitQuery=PER_OFFERS_SRCH_MAIN][]=&a1=&a2=30004989614 docviewer.yandex 0 0 [] [443,333,5760] [18] [] 0 0 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 2903448 0 0 0 1 0 0 117 2014-03-23T04:52:29 0 0 0 0 utf-8 330 0 0 1 9110769067793080479 287375675 0 0 0 1 0 E 2014-03-23T13:17:40 16 2 2 0 0 [] 1225767431 2672ee20d87b5c666121145dd76b3644cc 50936 -1 2 2W a5b3 0 1706 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7914288637035474585 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 62 1 -6867873495169505672 1 акте.ру - Лингво-лабора в Новостное авто виды спортал 1 2014-03-23T11:09 2014-03-23 1785 2750008691 d27e873642fdcc142fd52f628470ba33 241 7406071195777269826 1 156 122 http://91111478bcf-ff0001237248a3c684a84763.html http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/view_intext=картинки&clid=1&search.xml?hid=913017&l=map,trfe&trana.ru/23900/?&p=168603021-anons.html?ia=M&lf=F&af=35&s_c=3159_3296_16510865;0;1&x-email.yandsearch/?page=38.03.2014%2F1gEqdtN3LcNMB6CIKsO9mnONRBOu8roZ08buBPgjjNnbUGoe9Y5D3l9VbWj2D9Bt7erMicDOo%3D0%26fh_stream.ru/auto.yandex.ru;yandsearch;web;;%2Fwww.ivi.ru/filmId=rt.ru/saray.html?type_gruzoviki_i_kottelmesi/8926593&group_id=&stribuna-serial/1123413&text=доналдс расписьмо растен пропетро палом лечебный лады в газа коротоколаев метричек фото&nl=4&sp=1&target=2#12 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 13 8w 1 1 0 0 2723983 3 3 puma oxygen sex big tits porno 0 0 1997 547 157 2014-03-23T21:25:37 0 0 0 0 utf-8 401 0 0 0 8546017116003926276 709059014 0 0 0 0 0 E 2014-03-23T21:44:20 22 2 2 0 0 [] 2771758996 1c0ec512cf36858257e85c74d2ac7654a2 -1 -1 -1 nD a5b3 0 -1 0 61 100 108 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 9421855928520292696 11319102890696173680 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 5 0 -6880179944191362326 1 В пробегом - катеристочник 1 2014-03-23T16:12:53 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T05:48:42 0 0 0 0 utf-8 401 0 0 0 0 847205448 0 0 0 0 0 E 2014-03-24T07:51:46 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 95 93 793 0 5720 0 -1 -1 -1 5017 3724 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 0 -6986728922972248352 0 Женская библи в шопогрузки 1 2014-03-23T18:17:14 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/forums.kuba.com/kampaign/files http://yandex.net/ru/video/search yandex.ru tury.ru.msn 0 0 [85,350,1212,3849,8358] [106,323,1092,3676,13188] [18] [28] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 -1 0 0 0 592 503 117 2014-03-23T19:18:45 0 0 0 0 utf-8 401 0 0 0 0 1070533242 0 0 0 0 0 E 2014-03-23T12:25:44 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 1477 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6372803000235980495 925019927264479664 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -6998847403379936884 0 1 2014-03-23T21:14:59 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinopoisk.yandex.ua/Topic59953434017847910/news%2F2014&is_vative=0&searchplus.net%2Fwp tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:45:42 0 0 0 0 utf-8 330 0 0 1 0 606583569 0 0 0 1 0 E 2014-03-23T11:15:48 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 6 nD Tp 0 460 -1 -1 -1 -1 -1 -1 -1 -1 6631 5214 942 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 11446136684598192240 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 584 1 -7253547937523001925 1 Чтений найдено в Яндекс.Музыка" - Свежие сотказать бу авторов 1 2014-03-23T05:29:05 2014-03-23 1785 1739349077 d1e5e976bfd136ed9cad4fdb3524268e 102 1930034221481539513 1 156 4 http://911114784E20437b406b71056-mon-e-n-s-mesjachok http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 1088451 3 2 0 0 2011 877 945 2014-03-23T22:25:39 0 0 0 0 utf-8 401 0 0 0 5594739182691462682 576508429 0 0 0 0 0 E 2014-03-23T17:26:38 22 2 3 0 0 [2,1019,3993,9,14] 1650083551 0b32fa88d354c65c66315c662d206ac906b2 -1 -1 -1 nD a5b3 0 -1 9 172 243 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 13193074755622332528 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7395638780722795911 0 Современа - Футбольших 1 2014-03-23T16:23:10 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1830520[]=2&FILTER[32685626%2Fb%2Ffotogram/18 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20udWElM0QlM0RmZVNmZ1NNVzJyZkNmxFafzTGrazFsbKeQXfHEFzFfLycbGiwgeyVqAFM8Hj0hFn56d1xSX0ZOcwkxX0V5fAZR games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:49 0 0 0 0 utf-8 401 0 0 0 0 1015390617 0 0 0 0 0 E 2014-03-24T07:52:02 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15232020126713526616 11875920749657778288 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 122 0 -7407752634189615759 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:57:42 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,27,Zhitovit.com/iframe http://yandex%2F15551&lr=76&text=обезьянка haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:44 0 0 0 0 utf-8 401 0 0 0 0 663421629 0 0 0 0 0 c 2014-03-24T07:38:25 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 17131 -1 2 wi a5b3 0 -1 0 0 52 15 0 0 272 0 -1 -1 -1 401 -1 -1 0 0 07d21f 0 [] 0 713690030010978492 14503036338486044784 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -7479382069659086597 0 Продаю бесплатно - светике Татья владельцев 1 2014-03-23T21:20:45 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://kliksa.com/iframewom http://galerid=0&high-anner.by/?search?fr=all&text=купить доме&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgo moniac.com.travellecomme exist.ru.msn.com 0 0 [292,353,7606] [5760] [18] [28] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 5 0 0 0 430 384 117 2014-03-23T05:08:11 0 0 0 0 utf-8 330 1 0 0 5177399025069094312 567587409 0 0 0 1 0 E 2014-03-23T13:28:39 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 2843856621567445410 2250008690469188482 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 98 1 -7516343859483962604 1 Автомобили с проблетов онлайн на 1 2014-03-23T05:48:22 2014-03-23 22447063 3721844867 7e52689e524f80aac08d88e47e84e73d 9580 2301018798038122137 0 56 4 http://images.yandshijj-v-cheap-and http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1297 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3238011 3 2 0 0 653 693 117 2014-03-23T07:41:35 0 0 0 0 utf-8 330 0 0 0 4627722567800183582 485309851 0 0 0 0 0 E 2014-03-23T04:08:48 16 2 2 0 0 [] 3446251501 53d889ddb55859ed8adbe55c5cda7da177 45687 -1 1 wi a5b3 0 -1 17 14 261 21 0 0 1117 171 -1 -1 -1 2495 -1 -1 0 0 07d21f 0 [] 0 4313994922287858055 1100934878990743947 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7580359850907242723 0 продажа консультате с руков на AVITO.ru - страни в портименко 1 2014-03-23T19:23:45 2014-03-23 12591643 3565819545 82af9c35e16ca87ffaf7b9edfa6f42f6 239 2189462870885553765 1 97 42 https://mptri.com.tr/tr/videobox.tv/eventsnazorva/sevastok_2838##name=Новоставить http://yonja.com/etgifrm%2Ftitle,pos,p0,source,web,pre,site_slotname=Tümü&e_baski-gazetable_key=&price_valka_277565/?from=0m7g&clid=1965}/catalog%2Fsocial_hastnye-prizatsii/adme.ru/?clid=1&metrikansii/bez_opyta_sport-expresoriends.ru/ru/clck/jsredir?from=29.03.251629804b21hR1gwMGZyU013JTNEZnZTTVRBJTNEZmRTTVUUT2gtZEJhd1hHJKEMXiKRz8iFPth adultmultiki yandex.ru 0 0 [353,3121,11503,11630] [9,377,480,3676,15216,14241] [18] [28] 968 1640 57 0 0 0 0 7 73d7 1 1 1 1 LG Optimus 2853155 3 2 don шнуровногорий шёлк щёлк 0 0 592 547 296 2014-03-24T07:46:25 0 0 0 0 utf-8 401 0 0 0 8608824822004667778 816585860 0 0 0 0 0 E 2014-03-23T11:06:01 22 2 2 106 31 [397,11,9,68,14,1019,1465,1,3993,252,2] 3702300762 a18b5c7211334fac99217724417d4550d8 -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 17339020080848845144 8629326841291084175 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7599913264897309014 1 Отступным низкой области в Киеве. Решение пенсиональный фото доставни.ру — Яндекс.Музыке. Мне на AVITO.ru - Продаж - визажиганы - Мы от Good, B-russianFood.com: новости на новости на 20 формация 1000005 года, долла бывший чай - НОВОСТИ - 20 купить бесплатный с персонские первый заявки деловек птичка 1 2014-03-23T12:22:58 2014-03-23 1785 1353429726 e2c0e129718aad4e93f93b5c5c066ed675 3 6510982234989222954 1 105 7 http://9111147834-video-domasma-hd.net/best-1182761198 http://yandsearch;web;;%2Fwwwww.silverdi-kading/sex/page=310&text=фильмы онлайн штор&client_id=599277a696313/album/17-yil-hake.ru/?/ coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 3701 1667 29 8 0 0. 0 0 5 qR 1 1 0 0 2765439 3 2 0 0 2793 1041 157 2014-03-23T20:25:05 0 0 0 0 utf-8 401 0 0 0 9043742639069289622 56687838 0 0 0 0 0 E 2014-03-23T22:58:27 16 1 2 0 0 [] 1764751978 72253090507ab4406a6d83717c75243e -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 13306854087717566808 2241902997682127110 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7687055982568657638 1 Историентов на AVITO.ru - Поттехнику в асоціальный сайт На юго-востинга" : один бытового парфюмерили новости, дереводы. Realty.dmir.ru - Почта Mail 1 2014-03-23T16:58:38 2014-03-23 33044186 2126080452 fab776b7bdcf8dcb68cb6922724a9362 89 2034549784946942048 1 3 5 http://yandex.ru/cgi/online.com.ua/detskie-spb.blizko.ru/index.ru https://market.yandsearch/non.ru top-androeed rabota.yandex 0 0 [3,15,63,95,75,381,2182,4132,10886,16137,15261,13770] [2,15,46,123,102,507,3498,4504,15216,10157,15095,11161] [18] [28] 1846 1640 57 8 0 0.77 3 6 10 73d7 1 1 0 0 2175425 -1 0 0 0 1071 955 945 2014-03-24T11:26:33 5 1 19832 0 utf-8 401 0 0 0 5669772058100890221 888201737 0 0 0 0 0 E 2014-03-23T06:10:57 39 1 3 8971 0 [95,6,408,103,10,3,44,42,76,47,3993,209,51] 1341355226 fdd940e1982462885d4c8fb848816939 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15349291019717589336 8037358779388095153 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 0 0 -7780035710352963112 1 Голая Нагородской областической 1 2014-03-23T05:23:08 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/load.ru/#hs http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T05:29:56 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 546201586 0 0 0 0 0 5 2014-03-23T03:47:32 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 3 wi a5b3 0 -1 0 0 277 84 0 0 858 85 -1 -1 -1 1454 994 -1 0 0 07d21f 0 [] 0 4313994922287858055 18236259282679687280 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 10 0 -7790500540215037749 1 Тимошенковске на F1news: Новгородажа легководителей и ее европы - Странспорт 1 2014-03-23T10:51:56 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.139517&quickpantinoyleri http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 943 296 2014-03-23T22:22:46 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 515139757 0 0 0 0 0 E 2014-03-23T20:22:37 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 25 116 83 126 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 7982945165681148628 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 0 -7794706186709683946 1 В пробегом - катеристочник 1 2014-03-23T16:32:20 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:59:55 0 0 0 0 utf-8 401 0 0 0 0 667659827 0 0 0 0 0 E 2014-03-24T08:03:47 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 49 20 137 36 0 0 1861 0 -1 -1 -1 1987 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 236 0 -7829278250573826232 1 Продажа подержания найдено в Яндекс: нашлось 15 минут в интернет 1 2014-03-23T15:56:49 2014-03-23 27058368 1191418004 5c5ca181d9f05881a8e93d021a66fa5955 15887 372595447767099963 0 42 105 http://odnoklass_p112,113,14,15,2718-kak-dagitim.com%2FiHcpR_KXxms http://yandex%2F1538512 haberler.ru.msn.com.ua avito.ruel.mynet.com.tradeasure 0 0 [3302] [3666] [903] [996] 322 628 57 0 0 0 0 44 73d7 1 1 1 7 iPhone 2 4002484 -1 0 0 0 592 1990 117 2014-03-23T05:32:02 0 0 0 0 utf-8 401 0 0 0 0 708852166 0 0 0 0 0 E 2014-03-24T07:37:21 16 2 2 0 0 [] 1321231243 4ac1f0050169d628a611e12e66743079 22279 -1 1 wi a5b3 0 -1 0 0 1 28 0 0 559 0 -1 -1 -1 1476 -1 -1 0 0 07d21f 0 [] 0 16552074410276436312 849739178092755580 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -7893175662663208254 0 1 2014-03-23T21:20:38 2014-03-23 16862487 1542642122 17d7abdd4e5c6e12e9e0176ac55fa568d0 8363 1032885585239676511 0 111 71 http://e.mail.yandex.php?id_n=113993&Lt=1&carfiles/2014/9119&redirektor_view/27409,221&msid docviewer.yandex 0 0 [] [443,333,5760] [18] [] 317 296 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3229520 0 0 0 1 430 384 117 2014-03-23T05:08:04 0 0 0 0 utf-8 330 0 0 1 5177399025069094312 567587409 0 0 0 1 0 E 2014-03-23T13:28:33 16 2 2 0 0 [] 1497451434 fe0249b15f6ed7a5ce6891314f334a03 42333 -1 2 2W a5b3 0 314 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4149647501617670018 1104301 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 116 1 -7939467002160261687 0 1 2014-03-23T21:12:27 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogo_baskalabs.html tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:43:02 0 0 0 0 utf-8 330 0 0 1 0 529683494 0 0 0 1 0 E 2014-03-23T11:12:54 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 1 nD Tp 0 255 -1 -1 -1 -1 -1 -1 -1 -1 2253 3940 48 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 2665691200259067948 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 993 1 -7940922169703095731 0 Знакомства в Республике Дагестанцам 1 2014-03-23T11:36:25 2014-03-23 1785 994141745 9da519855c6298ca2b15a5da579246a283 207 1836964949227567248 1 42 71 http://9111147851/?&p=1&text=укладоватки закон http://yonja.com/kamuajanssories.ru/ulya-volt.ru/kino-haber.com/peugeot/230000&sp=-1&o=likest.tanka.pro/calendi-shpartaming%2Fsimferotiv-vosti.ua/oldcars coccoc.com yandex.ru 0 0 [3,5,6,15,57,91,76,175,353,403,349,3121,11503,11630] [119,3418,2455,3036] [18] [155] 1009 1367 57 0 0 0 0 0 73d7 1 1 1 92 iPad 7 1443169 3 2 психологда тура кирпич 0 0 674 1115 117 2014-03-23T19:40:39 0 0 0 0 utf-8 401 0 0 0 0 732369666 0 0 0 0 0 E 2014-03-23T02:32:34 55 2 2 0 0 [9,11,42,3,1] 990921201 f7aaf7453dcba45c62882493cd80e6fe81 -1 -1 -1 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 14359677112441125208 16829396780134885488 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7960500408286544976 0 Дикие стал автомобильники | каталог холстук, баз для мультуры по которы, услуг Россию :: Ходческие, фильм "Холодная рабочий купить 1 2014-03-23T21:20:31 2014-03-23 22422732 2271852515 56b72cde5096c0f237452cd9f670d92c 196 8579128732000753997 1 223 42 http://video/torre.com/odnoi-stimeout=1440&is_vative=0&s http://fast-topic/282039.html%3Fid%2F1000/query.ykt.ru/yandex.ru/fblogs.yandex.ru;yandsearch ficbook.net mail.yandex.ru 0 0 [5,4,372,9487] [15] [] [28] 968 1640 57 0 0 0 0 4 73d7 1 1 1 1 LG Optimus 1769215 1 0 0 0 592 838 157 2014-03-23T06:41:05 0 0 0 0 utf-8 401 0 0 0 4898301856994370214 460298661 0 0 0 0 0 E 2014-03-23T13:28:03 16 1 3 3918 6 [64,7,6,109,14,5,285,84,81,80,4] 2295410265 b140e0077981e3689a5f6973035a3b7e -1 -1 -1 nD Tp 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 10671434186064935256 1759711880979997785 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -7972070184216637013 0 Honda Bellas - Jerry - моя стулья | Новини 1 2014-03-23T21:12:45 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/bagajd-world/567765647 http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:43:26 0 0 0 0 utf-8 330 0 0 0 0 274009037 0 0 0 0 0 5 2014-03-23T11:13:16 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 132 174 11 0 462 6 -1 -1 -1 543 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 2948755971896419986 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 656 0 -8035654727346356734 1 В пробегом - катеристочник 1 2014-03-23T16:27:45 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yaristic_im.html mail.yandex.ru 0 0 [] [3666] [903] [] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 0 0 0 0 653 504 117 2014-03-23T06:02:54 0 0 0 0 utf-8 401 0 0 0 0 426221199 0 0 0 0 0 E 2014-03-24T08:04:20 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 0 0 46 119 0 0 740 0 -1 -1 -1 1412 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -8075310065356086365 0 Hotel.jpg» на продаю карте | | Для домов.НЕТ - поиск, познавала «Весен 1 2014-03-23T21:15:10 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/ball/anasayfa.irr.ru/page=0&availaut/to/casino http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:45:57 0 0 0 0 utf-8 330 0 0 0 0 11121749 0 0 0 0 0 5 2014-03-23T11:15:56 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 927 314 11 0 282 13 -1 -1 -1 1159 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 15729620011142801520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 658 0 -8094433118608774279 0 Светские главные, долла: упражные сериалы доллар - Страница 10 сон. Женскую система Rezervative Burcundai Sanatik 1 2014-03-23T16:33:25 2014-03-23 18645263 1790084893 60057a4e582b5faf88f2c49235bc138a 8363 803723286935347333 0 42 71 http://blog/resim-6/ http://yonja.com/quests forum.shestvennik.ru yandex.ru 0 0 [353] [] [] [28] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3031088 1 0 0 0 592 1376 117 2014-03-23T07:39:47 0 0 0 0 utf-8 401 0 0 0 5718608483174516921 922710799 0 0 0 0 0 E 2014-03-24T01:25:47 16 2 2 0 0 [] 1983574728 e3ebee64fe91325ffa29ba8579b9d3ae 62835 -1 1 wi I7 0 -1 0 0 0 7617 0 0 5486 984 -1 -1 -1 15108 -1 -1 0 0 07d21f 0 [] 0 2351151346570785378 7780118309660496456 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8149355527847772674 1 В пробегом - катеристочник 1 2014-03-23T16:24:43 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:54:01 0 0 0 0 utf-8 401 0 0 0 0 778610602 0 0 0 0 0 E 2014-03-24T07:58:46 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 16 22 61 62 0 0 785 0 -1 -1 -1 1064 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 173 0 -8155806667442330296 0 Hotellot - Рыбалка.ру - все серия 1. Писточный или Турция, Гомельной банка 1 2014-03-23T21:14:08 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/ball/image&uinfo=ww-135136361 http://women/way4.docx&c=532e95d64c751729f&keyword-of-the-bagisa-anglando.ua/yandex.ru/myhard.mobili_s_probegom/contestoriches tour.com.tr megas.russia.ru 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 -1 0 0 0 412 567 296 2014-03-23T06:44:44 0 0 0 0 utf-8 330 0 0 0 0 44551634 0 0 0 0 0 5 2014-03-23T11:14:53 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 3 nD Tp 0 -1 0 0 184 499 18 0 779 7 -1 -1 -1 1948 -1 -1 0 0 07d21f 0 [2170618] 0 11446136685140392280 5463880999052126868 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 823 0 -8156909267415167339 1 Брюки с TurboBit.net - фильмы онлайн на 2014 - Финансий 1 2014-03-23T16:23:08 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://hurpass.com/ogrevolyni http://foto-395077193b7240786&lr=11110436606 photostrana cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [9,121] [] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:47:01 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 845095662 0 0 0 1 0 E 2014-03-23T23:47:08 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 51 2009 1293 2 702 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 18315239292404426864 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 7 1 -8157755090657987777 1 вк — Яндекс.Почта/АndroidMag 1 2014-03-23T09:50:56 2014-03-23 1785 46316870 02ae30409714778e47373be3a09c3737 71 1806327137114274628 1 79 4 http://9111147842,0.01178.1401/sovmestore http://goodanies/sent-gamesearch coccoc.com forums.ruvr.ru 0 0 [1,2,5,4,6,7,9,17,16,56,126,353,477,3324,3849,3121,6308,6479,8856,11503,13770,14847,11630] [119,3418,2455,3036] [18] [352,267,694] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 1284488 3 2 0 0 656 637 296 2014-03-23T21:17:04 0 0 0 0 utf-8 330 0 0 0 9067757825491233857 834077893 0 0 0 0 0 E 2014-03-23T19:47:34 26 2 2 399 0 [7,1011,11,93,6,501,105,9,14,2938,3528,2] 45909282 fb7b38cc53716a5bae6358aad8823063 -1 -1 -1 nD a5b3 0 -1 11 77 91 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6922718891101119046 9450665378893719664 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -8165890628220817396 0 1 2014-03-23T21:13 2014-03-23 30721346 2136809579 2281f9b7ff39949fb0212c8f199c8571 189 9655497281435683907 0 42 58 http://kinogue.ru/213/bagaji-demoi-fena tour.com.tr 0 0 [] [] [] [] 1012 1980 24 0 0 0 0 23 73d7 1 1 1 0 3134501 0 0 0 1 412 567 296 2014-03-23T06:43:36 0 0 0 0 utf-8 330 0 0 1 0 274009037 0 0 0 1 0 E 2014-03-23T11:13:27 39 2 3 119 0 [3,42,14,70,76,72,10,925,103,1] 1850966214 0fe889e10f7013af6d68bb0680f79f76 1974 -1 5 nD Tp 0 204 -1 -1 -1 -1 -1 -1 -1 -1 2055 3158 320 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 855065554003365461 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 766 1 -8207621839602888071 0 21:24:02) « Политик 1 2014-03-23T18:06:44 2014-03-23 1946 2556599157 df5749fc6f072d53196cecf7b048918e 114 551112348121871248 1 42 71 http://avito.ru/for/zilcc.ru/tsotsbank yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 1009 1980 57 0 0 0 0 6 73d7 1 1 1 7 2289966 0 0 0 0 430 234 117 2014-03-23T19:08:37 0 0 0 0 utf-8 401 0 0 0 0 131924827 0 0 0 0 0 E 2014-03-23T12:13:53 0 0 0 0 0 [] 3345370891 157b8bf1d874256692af07593777dd86 -1 -1 -1 HX J4 0 -1 0 0 0 3912 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 6785386778629335136 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 6 0 -8262700426099359718 1 1 2014-03-23T16:21:13 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://video/embed/68411756114414293 http://foto-395077193b7240786&lr=11110436606 amksports.ru.livemaster cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [45,333] [18,348] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:44:38 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 42215400 0 0 0 1 0 E 2014-03-23T23:45 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 49 5647 4852 2 2891 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 2532179236054953957 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8321182583543853388 0 1 2014-03-23T20:34:29 2014-03-23 24727537 3731398286 f8655b8768c543560ef532c64cb25566 11729 2192651662885533197 0 56 3 http://ads.search?clid=20836950/lodosnabworks/991168&t=25 tranamaschools 0 0 [] [] [] [] 1297 555 29 8 0 0. 0 0 26 73d7 1 1 0 0 1 0 0 0 1 1 1 117 2014-03-23T06:05:11 0 0 0 0 utf-8 401 0 0 1 0 122853544 0 0 0 1 0 E 2014-03-24T14:17:35 0 0 0 0 0 [] 3764369569 55f84d6dba4d0103743b9f798c7d6437 -1 -1 -1 2W Ti 0 2998 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 14518813701131921520 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8456596711470618355 0 Samsung Galaxy tarisme online Search resmi Sistem TurboBit.net.ua - DX 130 000 рубля игры 1 2014-03-23T06:47:03 2014-03-23 1785 901953317 a4a276d0ea345c74148750a6919a36dad5 207 1836964949227567248 1 42 71 http://rutube.com.tr%2Fgazpromo38.ru/search?lr=2&rpt=simage avito.russia.rust.net.ru 0 0 [] [3,160,864,2455] [] [] 1009 795 57 0 0 0 0 0 73d7 1 1 1 92 iPad 7 1443169 0 0 0 0 674 927 117 2014-03-23T14:25:27 0 0 0 0 utf-8 401 0 0 0 0 833653080 0 0 0 0 0 E 2014-03-23T19:03:13 55 2 2 0 0 [9,11,42,3,1] 1061339287 20bfc7e7245325fdac7e99f96f7dcbe6 -1 -1 -1 2W a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 13975993063278569584 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -8472838923458193599 0 Личный компаний: "В бороны панели 1 2014-03-23T10:26:41 2014-03-23 1946 2822174503 83f4c4017c625c30615e808b9b553cd25a 950 1989946518134869356 1 42 71 http://avito.ru/shoes-with-avciliusersView yandex.ru 0 0 [] [106,323,1092,3676,13188] [18] [] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 1 0 0 0 0 592 1376 352 2014-03-23T04:42:49 0 0 0 0 utf-8 401 0 0 0 0 285947293 0 0 0 0 0 E 2014-03-23T02:39:02 16 2 2 66 0 [2193,82,265,14,95,6,100,9,72,3275,1,70,11] 3219490004 5c6e7add6158bbed0699cbe973042ea2ef -1 -1 -1 nD Tp 0 -1 0 0 0 2817 2921 2527 30475 0 -1 -1 -1 29211 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 15977284023899318384 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 15 0 -8555937822471823535 0 1 2014-03-23T16:34:48 2014-03-23 18645263 1790084893 60057a4e582b5faf88f2c49235bc138a 8363 803723286935347333 0 42 71 http://blog/resim-6/ forum.shestvennik.ru 0 0 [] [] [] [] 1012 1980 57 0 0 0 0 6 73d7 1 1 1 7 iPhone 2 3031088 0 0 0 1 592 1376 117 2014-03-23T07:41:33 0 0 0 0 utf-8 401 0 0 1 5718608483174516921 922710799 0 0 0 1 0 E 2014-03-24T01:27:26 16 2 2 0 0 [] 1983574728 e3ebee64fe91325ffa29ba8579b9d3ae 62835 -1 1 wi I7 0 2002 -1 -1 -1 -1 -1 -1 -1 -1 25844 25608 85 -1 -1 -1 0 0 07d21f 0 [] 0 15284527577228392792 7780118309660496456 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 1 -8563276039202285702 1 Голос видео 1 2014-03-23T05:23:15 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/triket/4746734017811fc051dd0e57 http://yandex.ru;yandex.ru/?area=ShowForum.ru/look/event cdo.e1.ru.plays олины.рф 0 0 [] [] [] [] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 -1 0 0 0 1460 549 574 2014-03-24T05:30:03 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 215720129 0 0 0 0 0 5 2014-03-23T03:47:38 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 28468 -1 5 wi a5b3 0 -1 0 0 289 1 0 0 94 20 -1 -1 -1 488 414 -1 0 0 07d21f 0 [] 0 18236259283221887320 16190469452247787632 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 9 0 -8599500047480121116 1 Брюки с TurboBit.net - фильмы онлайн на 2014 - Финансий 1 2014-03-23T16:22:36 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://hurpass.com/ogrevolyni http://foto-395077193b7240786&lr=11110436606 photostrana cars.auto.ru.msn 0 0 [9,8,193,2199,3849] [9,121] [] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 5 0 0 0 1994 561 117 2014-03-23T06:46:24 0 0 0 0 utf-8 401 1 0 0 9007862211105217806 42215400 0 0 0 1 0 E 2014-03-23T23:46:39 16 2 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8973952666555335426 18315239292404426864 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 3 1 -8602651835226449867 1 7 самая мамки для ваше может по суперма Страница 719 тыс. ответов 1 2014-03-23T12:16:45 2014-03-23 1785 2088542490 921202579dbab4e58eddb04f693854b3 57 9499572962667875395 1 56 122 http://911114788/38/celebekgaleri.hurriyet http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/downloadTo=&orderby=RelAmountry_id=22&ved=0CAUQjB0&url=http://money?rand_cruitstart=444305.952058.13951639370f280c133ad16ce2c79e7cab93c5a23X3 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1339 555 57 8 0 0. 0 0 11 6ee9 1 1 0 0 2366248 3 3 схема сумерки 5-6 летовой устанния 0 0 1627 708 157 2014-03-23T22:11:02 5 1 24018 0 utf-8 401 0 0 0 8929720244896745512 460839648 0 0 0 0 0 E 2014-03-23T22:21:38 0 0 0 0 0 [] 114378192 d7364e6a7ddcbcf6dcccca7bd6b2807a -1 -1 -1 nD a5b3 0 -1 0 0 626 189 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 6784513853171268256 7631625543297538199 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8607066140185696713 1 Отслеживаны - №1065 - ekşi 1 2014-03-23T16:28:07 2014-03-23 12461549 2042816759 b7500e7877917372f6e4642b5733ffa2 8363 2084898566169119758 0 3 4 http://yandex.ru/yandex.kz/weatherson.html#78efe9869.shtml?wmid=143&srcid=UxB0pAAA6I9d0CWaWE%3DfrSMw http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1297 1010 29 8 0 0. 0 0 44 73d7 1 1 0 0 2938383 -1 0 0 0 653 504 117 2014-03-23T06:03:11 0 0 0 0 utf-8 401 0 0 0 0 813102074 0 0 0 0 0 E 2014-03-24T08:04:40 0 0 0 0 0 [] 2080447566 a98e68908c21690b80925c62f15a6785ce -1 -1 -1 wi a5b3 0 -1 15 110 78 102 0 0 2060 365 -1 -1 -1 2397 1361 -1 0 0 07d21f 0 [1555356] 0 6331778101536488133 11618756116771170416 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 8 0 -8611399304547034181 0 Современа - Футбольших 1 2014-03-23T16:22:51 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=1188c56ff8058343682.1_29_et._30108879a9aa61ea73752719 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20vdmlyLnJ1Lw%3D%26engine_volume2=40&sid=6&ved games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:23 0 0 0 0 utf-8 401 0 0 0 0 356217277 0 0 0 0 0 E 2014-03-24T07:51:41 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 11767694963832011096 2937255180427836822 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 111 0 -8664317093542350977 1 Афиша - купить готовой менение сиUçuşlardo, Yazlık Burçlar - astopol.info - Classion AAAQ (250Hp) CVT (140 тыс. км., смотр объявлению в автобусов - Екатегория России, клиент 1 2014-03-23T20:34:25 2014-03-23 32821467 1157253311 482b48fd303d64bba86ec7965c6ed6dd67 207 2460387875241374418 0 56 4 http://rabota.ru/show%2F2014 http://read.php?act=8&ved=0CC85ViwdY8L4WG9pLWlKS0RJU3AeAxJuh wwww2.sports m.auto.yandex 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3815009 -1 0 0 0 1136 593 117 2014-03-23T06:05:10 6 0 39566 0 utf-8 330 0 0 0 7635035874316412186 1000586617 0 0 0 0 0 5 2014-03-24T14:17:28 39 1 2 70 51 [10,42,1,22,3,112,3773,103,5,51,15,4,3212,47] 1549383847 ed3b609e3397b12c56cca4469c881c38 17234 -1 7 wr a5b3 0 -1 0 0 179 57 0 0 212 103 -1 -1 -1 613 -1 -1 0 0 07d21f 0 [] 0 3832538266798636143 15412096237897967728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 22 0 -8665395302091187491 0 Современа - Футбольших 1 2014-03-23T16:22:43 2014-03-23 12461549 1575473191 6d797a1cbedbd631aa4d8587302bbd63 15887 16742594366701915203 0 223 42 https://mail.yandex.ru/dage-olume[2]=&engine_volum-yenisafak.com.tr%2Fsite_id=106/cid=191&rpt=simages%2Fthumb%2Fimage&lr=157 http://vestivarka=39&model.xml?type=11&ved=0CCgQFjAJ&url=aHR0cDovL21hcmtldGdpZC5jb20v&com_id=0&body_type=0&clid=1980&s_tg=&tariniz games.mail.yandex vk.com.ua-footoday 0 0 [3302] [3666] [903] [996] 317 763 57 0 0 0 0 0 73d7 1 1 1 1 S820_ROW 2699355 -1 0 0 0 430 407 117 2014-03-23T05:45:12 0 0 0 0 utf-8 401 0 0 0 0 588577730 0 0 0 0 0 E 2014-03-24T07:51:28 0 0 0 0 0 [] 1929084301 8cbe35d18ebe77de6693f8124f8eeb90 -1 -1 -1 wi I7 0 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 1560098688466543352 11767694963289811056 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 121 0 -8673158852214232182 1 Голая Нагородской областической 1 2014-03-23T03:51:08 2014-03-23 22447063 1227639417 59e51dbfbfbb72edccc1054e4856c8fa 116797 626016583650557966 0 3 4 http://images.yandex.ru/load.ru/#hs http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 0 [353] [] [] [28] 1846 1640 29 8 0 0. 0 0 44 73d7 1 1 0 0 3683672 3 2 0 0 1460 549 574 2014-03-24T03:26:28 5 1 19832 0 utf-8 401 0 0 0 7080158683485407478 270908049 0 0 0 0 0 E 2014-03-23T02:24:56 16 2 2 0 0 [] 1088914233 be30ca76a813635c748c771bf336305a50 55446 -1 3 wi a5b3 0 -1 31 93 484 10 0 0 546 176 -1 -1 -1 1135 -1 -1 0 0 07d21f 0 [] 0 4313994922287858055 18236259282679687280 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 4 0 -8725981425210147796 0 Аудио ЭХО Москве, ножений вопросова нашлось 397 1 2014-03-23T02:41:19 2014-03-23 15665476 1648690379 04eb9547b256147702f34bd2464beaf0 196 1425686864647023563 1 56 3 http://ivona.ru/top.rbc.ru/moscow=&q= Erection27=&toName=false&morpholog/vakansition&ref=city/daily http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/news.rambler.ru/search/ex09KBBcpWRDlaYTZYNEJHZDlOb3VHdjNWRXg5UFBpV2pFQ1qUQ&b64e=2&output=uds_app_id=19855479&view?ad=44&s_yers.don.ru/context=Sex&geo=0&s_tg=&offset=2.\\tПоповорого план кубе&clid=20639&keyno=0&l10n=ru&mc=1.58496250001978934&Lt=6&refresh=1&has_photofunia.ru/#!/video/search;web;;%2Fweb%2Fitem%2Ftitle,pos,p1,source=web&text=текстра фото 2014/view/125329d08a194e758644-500h_bWOg&bvm=bv.62922401,d.bGE&cad=rjt market.ru.platnye-igrydlja tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [] [] [28,267] 1622 1583 29 8 0 0. 0 0 10 73d7 1 1 0 0 1808122 3 3 интернет магазин финес и физическу о приколеснока, валерия 0 0 2011 768 157 2014-03-23T04:34:29 5 1 19832 0 utf-8 330 0 0 0 6243379005186084238 8777098 0 0 0 0 0 E 2014-03-23T21:50:32 16 2 3 854 0 [2193,82,265,3275,347,14,72,925,100,3,80] 1245455042 891bf70623c57bfdd5e9d09d616390eb -1 -1 -1 nD Tp 0 -1 0 72 206 84 12371 0 -1 0 -1 -1 -1 18 -1 -1 0 0 07d21f 0 [] 0 2560641399298933162 12328847313983957104 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8746280778383767298 1 Поздравоохранул 1 2014-03-23T13:55:17 2014-03-23 1785 3609570631 e9fd65b19c652c4ce85594f2fcd0c7db 196 593475812248875581 1 227 105 http://911114783342507357617c9eb279bb4f2642 http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/video/index.ua/pole-2167ba0f7c355305-nam-v-pid2-290622263757&m=3,5&Volume[1]=&extras[24]=0&eae=0&clid=22876.26.2.0.html&docid=onair.com/galeride/Kariy_el/zafiruet-onlajn_tr_5Gn0cFj_bANt_S8ROdvfrOg6pCgU5XY2P3MtHkZQNDILDhspeAAhWjJJgkFq5HXGrEC5GmjXQ5bGV3TTVeklJakNwelhKc0hDcjFTVTFWb180ff0d9&url=http:%2F%2Foktavito1296_3346d0f0;dcid=ya_wireleva.ru/yandex.ru/?action=page-67600&groups[]=17704&groups[]=13&msid=91cc2424241d2cd8128&lr=97541¬custom_key=&haggle_AO_Mondelevizit-dvigator.tv/tnt/?region=cadf2922401,d.bGE&cad=rjt coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 523 617 57 0 0 0 0 44 73d7 1 1 1 0 3684816 3 3 алекс старт 20кг купить десан автобазаречь создать 0 0 592 310 157 2014-03-23T22:38:38 0 0 0 0 utf-8 401 0 0 0 8382449486543033592 279598542 0 0 0 0 0 E 2014-03-23T03:43:12 16 1 3 119 4 [76,1] 3886548777 f69565e66e18aeaf2059a8b6495c723613 -1 -1 -1 nD a5b3 0 -1 0 28 22 9 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 4244945004353045279 5931335146677711795 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -8931522039698213139 1 В пробегом - катеристочник 1 2014-03-23T16:28:08 2014-03-23 12461549 1766508279 e59f66e9db9acda7f41e5d3e48b0c447 11534 314871572878047037 0 56 4 http://yandex.ru/yaristic_im.html http://mynet.com%2Ftitle,pos,p1 mail.yandex.ru naughtmyblaze.ru 0 0 [3302] [3666] [903] [996] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 4002484 -1 0 0 0 1802 942 117 2014-03-23T05:56:58 0 0 0 0 utf-8 401 0 0 0 0 126947969 0 0 0 0 0 E 2014-03-24T08:00:54 0 0 0 0 0 [] 1552086083 b05e8e1b999dea2fcc42b2e455837044 -1 -1 -1 wi a5b3 0 -1 22 23 62 61 0 0 996 0 -1 -1 -1 1043 -1 -1 0 0 07d21f 0 [] 0 6331778101536488133 4983133167646354697 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 240 0 -8933906015862790098 1 Афиша - купить готовой менение сиUçuşlardo, Yazlık Burçlar - astopol.info - Classion AAAQ (250Hp) CVT (140 Hp) 2006 купить компании Украина вытащил о Москвы :: Голосуток в 1 2014-03-23T20:34:36 2014-03-23 32821467 1157253311 482b48fd303d64bba86ec7965c6ed6dd67 207 2460387875241374418 0 56 4 http://rabota.ru/new.mcko http://read.ru/yandsearch wwww2.sports m.auto.yandex 0 0 [] [] [] [] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3815009 -1 0 0 0 1136 593 117 2014-03-23T06:05:24 6 0 39566 0 utf-8 330 0 0 0 7635035874316412186 28170112 0 0 0 0 0 5 2014-03-24T14:17:36 39 1 2 70 51 [10,42,1,22,3,112,3773,103,5,51,15,4,3212,47] 1549383847 ed3b609e3397b12c56cca4469c881c38 17234 -1 7 wr a5b3 0 -1 0 0 106 58 0 0 92 34 -1 -1 -1 148 -1 -1 0 0 07d21f 0 [] 0 15412096238440167768 13753603333183694960 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 19 0 -8939870126547196026 1 Комная прессан Партира, акциях — фанфики к экзаментально и части Донецка | РБК daily Digital acro 1 2014-03-23T16:02:25 2014-03-23 1785 2922543887 1c01b17e6ee3c1a01828ecd318a6b581 59 8723854704339558313 1 56 2 http://9111147842,0.01178.1394999-prepovedeki http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net%2F1b.jpg","photos&p=3&page=18413930_handguides[]=2&sign=cf8f01c12489e0a7&uuinfo/view/93190876323017&l=sem&sign=213&nore[2]=0&search?p=законы финальных треуголоваров владимирова туалеты 10 дневники андроид&lr=20525944&lr=1637&keyno=0&room=0&page=576&m[]=100&search/retsiatoust/avtobank.ru/#!/kemeye-karechenkoi-denegro/brando.ua%2F17%2F204a%2F&ei=0-ochekiev.ua/opinion%26wheelpUTdBQUhtbTZhSzItYVZOOUjhfNF9vZDdGVzLzAwMzA3L3BwLzQxNjY2ZHpnY2VWxiCHsuJClHPnYMQVoxbSVTUFJTElsakI4WFp6NHNHJRDRhOU1LR3BuSmJYYj9tOUmxUOXMtVlN1TjNtVGc1c032a1b3672 coccoc.com tanks.co.il 0 0 [1,2,5,4,6,7,9,10,15,30,27,56,65,117,81,122,102,162,232,353,392,1004,3536,3324,3849,4695] [119,3418,2455,3036] [18] [28,267] 1297 1010 57 8 0 0. 0 0 18 73d7 1 1 0 0 2992777 3 3 детский номер электрообная игры скорота фото кало 0 0 1366 391 157 2014-03-23T00:56:51 0 0 0 0 utf-8 330 0 0 0 4847654860042290847 969371126 0 0 0 0 0 E 2014-03-23T02:42:26 55 1 3 12257 0 [10,9,112,68,365,76,260,22,1,2817,3,3540] 2380200681 0735dfb043075c30693475487bf24a49b7 -1 -1 -1 nD a5b3 0 -1 1 55 84 -1 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 5849787649459211608 17662636599668303984 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 2 0 -8964700706069591280 1 Салата 10. Цензор видео, онлайн (все забилей с пройти в Мельного 1 2014-03-23T04:19:17 2014-03-23 22447063 1796001934 3ae86dda5556b7f4ef38aa8077f6825a 15887 1644674872295047497 0 3 4 http://images.yandex.ru/neo2/#inbox/&dates=0&run[1 http://go.mail.ru&js=1&ref cdo.e1.ru.plays yandex.ru 0 2 [353] [] [] [28] 1587 555 57 10 13 0.77 0 0 25 73d7 1 1 0 0 3996694 3 2 0 0 1713 555 117 2014-03-23T19:14:50 0 0 0 0 utf-8 401 0 0 0 0 939725270 0 0 0 0 0 E 2014-03-23T02:46:14 0 0 0 0 0 [] 1907295579 dc652869f8d8eff9ed5b5daa5374b163 5038 -1 3 wi a5b3 0 -1 305 0 452 30 0 0 4029 164 -1 -1 -1 5023 6719 -1 0 0 07d21f 0 [] 0 4313994922287858055 9730402382055663728 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -9065033574850035452 1 Серия 9 - 2013. Дети@Mail.Ru — социалист 1 2014-03-23T16:20:14 2014-03-23 8221043 3199985924 3fd3d57ddd80578caa3a01aeae456454 162 851600322817957779 0 56 4 http://yandsearch?lr=35&rpt=simage&lr=66&text=розаврилова http://google.com/games.mail.ru ista.uaprodessa cars.auto.ruel 0 0 [5,9,57,353,3849] [13,11,240,3918,3676] [18] [28] 1622 1583 29 8 0 0. 0 0 44 73d7 1 1 0 0 4076823 3 2 0 0 1994 561 117 2014-03-23T06:43:38 0 0 0 0 utf-8 401 0 0 0 9007862211105217806 42215400 0 0 0 0 0 E 2014-03-23T23:44:06 16 1 2 858 0 [14,5,80,4,6,72] 2532292687 e6d09df4e1d112d79e6bfe9bfb523b79 -1 -1 -1 nD a5b3 0 -1 47 40 590 134 0 0 2110 -1 -1 -1 -1 -1 3460 -1 0 0 07d21f 0 [] 0 1930741556500475038 9207967286803898539 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 -9186659792231785281 1 Настройствоваться частных (б/у) автор о продукции; развития детьми - Грузов 1 2014-03-23T09:48:08 2014-03-23 1785 1341248658 6366e773993d35514d6b846f79b34292 183 626923241082458450 1 56 4 http://9111147832977565.html?cat=420801;label=perioda http://go.mail.yandsearch coccoc.com yandex.ru 0 0 [3,5,6,15,57,91,76,175,353,403,349,3121,11503,11630] [119,3418,2455,3036] [18] [155] 1339 555 29 8 0 0. 0 0 44 73d7 1 1 0 0 3500889 3 2 0 0 1997 548 117 2014-03-23T17:20:17 0 0 0 0 utf-8 330 0 0 0 8416052423457712746 215289560 0 0 0 0 0 E 2014-03-23T23:14:07 16 2 2 13621 14 [14,72,80,925,370,109,7,285,3274,101,6,66] 1160820115 524ee8575739a6149a641e6f4fbc6f7b -1 -1 -1 nD a5b3 0 -1 9 45 109 42 0 0 -1 -1 -1 -1 -1 -1 -1 -1 0 0 07d21f 0 [] 0 8578740285396261239 4548538545130905100 0 0 [] [] [] [] [] \N c1889e2b9ad1e219ed04c0e9624b5139 1 0 - diff --git a/regression-test/data/mv_p0/ssb/multiple_no_where/multiple_no_where.out b/regression-test/data/mv_p0/ssb/multiple_no_where/multiple_no_where.out index 00e05ab400e7ca9..ceec2c123596c8a 100644 --- a/regression-test/data/mv_p0/ssb/multiple_no_where/multiple_no_where.out +++ b/regression-test/data/mv_p0/ssb/multiple_no_where/multiple_no_where.out @@ -22,3 +22,7 @@ ASIA ASIA 1992 1 0 nation 0 1993 nation 0 +-- !select_temp_2 -- +1 4 1 1 +2 8 2 2 + diff --git a/regression-test/data/mv_p0/test_dup_mv_div/test_dup_mv_div.out b/regression-test/data/mv_p0/test_dup_mv_div/test_dup_mv_div.out new file mode 100644 index 000000000000000..64c881ed10b0211 --- /dev/null +++ b/regression-test/data/mv_p0/test_dup_mv_div/test_dup_mv_div.out @@ -0,0 +1,13 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_star -- +-4 -4 -4 4 +1 1 1 1 +2 2 2 b +3 -3 \N c + +-- !select_mv -- +-4 -4.0 +1 1.0 +2 2.0 +3 -3.0 + diff --git a/regression-test/data/mv_p0/varchar_length/varchar_length.out b/regression-test/data/mv_p0/varchar_length/varchar_length.out new file mode 100644 index 000000000000000..d944f69b4a3c1a2 --- /dev/null +++ b/regression-test/data/mv_p0/varchar_length/varchar_length.out @@ -0,0 +1,8 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_exp -- +test1 UNIQUE_KEYS vid VARCHAR(1) VARCHAR(1) No true \N true + report_time INT INT No true \N true + +mv_test UNIQUE_KEYS mv_report_time INT INT No true \N true `report_time` + mv_vid VARCHAR(65533) VARCHAR(65533) No true \N REPLACE true `vid` + diff --git a/regression-test/data/nereids_function_p0/fn_test.dat b/regression-test/data/nereids_function_p0/fn_test.dat index 54877f74b51b4ea..097a849885d406c 100644 --- a/regression-test/data/nereids_function_p0/fn_test.dat +++ b/regression-test/data/nereids_function_p0/fn_test.dat @@ -1,13 +1,13 @@ -0;0;1;1;1;1;1;0.1;0.1;0.1;0.1;0.1;0.1;0.1;0.1;char11;char21;char31;varchar11;varchar21;varchar31;string1;2012-03-01;2012-03-01;2012-03-01 01:00:01;2012-03-01 01:00:01;2012-03-01 01:00:01;2012-03-01 01:00:01;[0];[0];[1];[1];[1];[1];[1];[0.1];[2012-03-01,2012-03-01];[2012-03-01 01:00:01];[2012-03-01,2012-03-01];[2012-03-01 01:00:01,2012-03-01 01:00:01,2012-03-01 01:00:01,2012-03-01 01:00:01];[char11,char21,char31];[char11,char21,char31,varchar11,varchar21,varchar31];[char21,char31,varchar11,varchar21,varchar31,string1];[0.1,0.1];ST_Point(12.1, 32.1);ST_Point(12.1, 32.1);5;5;116.4274406433;39.9020987219;0;1;4.321;LINESTRING (1 1, 2 2);POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0)) -1;0;2;2;2;2;2;0.2;0.2;0.2;0.2;0.2;0.2;0.2;0.2;char12;char22;char32;varchar12;varchar22;varchar32;string2;2012-03-02;2012-03-02;2012-03-02 02:01:02;2012-03-02 02:01:02;2012-03-02 02:01:02;2012-03-02 02:01:02;[0];[0];[2];[2];[2];[2];[2];[0.2];[2012-03-02,2012-03-02];[2012-03-02 02:01:02];[2012-03-02,2012-03-02];[2012-03-02 02:01:02,2012-03-02 02:01:02,2012-03-02 02:01:02,2012-03-02 02:01:02];[char12,char22,char32];[char12,char22,char32,varchar12,varchar22,varchar32];[char22,char32,varchar12,varchar22,varchar32,string2];[0.2,0.2];ST_Point(12.2, 32.2);ST_Point(12.2, 32.2);126.35620117;-39.939093;56.4274406433;66.9020987219;0;2;5.3;LINESTRING (1 1, 2 2);POLYGON ((1 1, 4 1, 4 4, 1 4, 1 1)) -2;0;3;3;3;3;3;0.3;0.3;0.3;0.3;0.3;0.3;0.3;0.3;char13;char23;char33;varchar13;varchar23;varchar33;string3;2012-03-03;2012-03-03;2012-03-03 03:02:03;2012-03-03 03:02:03;2012-03-03 03:02:03;2012-03-03 03:02:03;[0];[0];[3];[3];[3];[3];[3];[0.3];[2012-03-03,2012-03-03];[2012-03-03 03:02:03];[2012-03-03,2012-03-03];[2012-03-03 03:02:03,2012-03-03 03:02:03,2012-03-03 03:02:03,2012-03-03 03:02:03];[char13,char23,char33];[char13,char23,char33,varchar13,varchar23,varchar33];[char23,char33,varchar13,varchar23,varchar33,string3];[0.3,0.3];ST_Point(12.3, 32.3);ST_Point(12.3, 32.3);16.35620117;19.939093;76.4274406433;46.9020987219;0;3;7.321;LINESTRING (1 1, 2 2);POLYGON ((0 0, 12 0, 10 16, 1 10, 0 0)) -3;0;4;4;4;4;4;0.4;0.4;0.4;0.4;0.4;0.4;0.4;0.4;char11;char21;char31;varchar11;varchar21;varchar31;string1;2012-03-04;2012-03-04;2012-03-04 04:03:04;2012-03-04 04:03:04;2012-03-04 04:03:04;2012-03-04 04:03:04;[0];[0];[4];[4];[4];[4];[4];[0.4];[2012-03-04,2012-03-04];[2012-03-04 04:03:04];[2012-03-04,2012-03-04];[2012-03-04 04:03:04,2012-03-04 04:03:04,2012-03-04 04:03:04,2012-03-04 04:03:04];[char11,char21,char31];[char11,char21,char31,varchar11,varchar21,varchar31];[char21,char31,varchar11,varchar21,varchar31,string1];[0.4,0.4];ST_Point(12.4, 32.4);ST_Point(12.4, 32.4);-46.35620117;39.939093;23.4274406433;-26.9020987219;0;4;3.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 11 4, 42 44, 4 18, 1 1)) -4;0;5;5;5;5;5;0.5;0.5;0.5;0.5;0.5;0.5;0.5;0.5;char12;char22;char32;varchar12;varchar22;varchar32;string2;2012-03-05;2012-03-05;2012-03-05 05:04:05;2012-03-05 05:04:05;2012-03-05 05:04:05;2012-03-05 05:04:05;[0];[0];[5];[5];[5];[5];[5];[0.5];[2012-03-05,2012-03-05];[2012-03-05 05:04:05];[2012-03-05,2012-03-05];[2012-03-05 05:04:05,2012-03-05 05:04:05,2012-03-05 05:04:05,2012-03-05 05:04:05];[char12,char22,char32];[char12,char22,char32,varchar12,varchar22,varchar32];[char22,char32,varchar12,varchar22,varchar32,string2];[0.5,0.5];ST_Point(12.5, 32.5);ST_Point(12.5, 32.5);43.35620117;35.939093;35.4274406433;35.9020987219;0;5;2.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 16 1, 16 16, 1 16, 1 1)) -5;0;6;6;6;6;6;0.6;0.6;0.6;0.6;0.6;0.6;0.6;0.6;char13;char23;char33;varchar13;varchar23;varchar33;string3;2012-03-06;2012-03-06;2012-03-06 06:05:06;2012-03-06 06:05:06;2012-03-06 06:05:06;2012-03-06 06:05:06;[0];[0];[6];[6];[6];[6];[6];[0.6];[2012-03-06,2012-03-06];[2012-03-06 06:05:06];[2012-03-06,2012-03-06];[2012-03-06 06:05:06,2012-03-06 06:05:06,2012-03-06 06:05:06,2012-03-06 06:05:06];[char13,char23,char33];[char13,char23,char33,varchar13,varchar23,varchar33];[char23,char33,varchar13,varchar23,varchar33,string3];[0.6,0.6];ST_Point(12.6, 32.6);ST_Point(12.6, 32.6);47.35620117;26.939093;47.4274406433;75.9020987219;0;6;33.321;LINESTRING (1 1, 2 2);POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4)) -6;0;7;7;7;7;7;0.7;0.7;0.7;0.7;0.7;0.7;0.7;0.7;char11;char21;char31;varchar11;varchar21;varchar31;string1;2012-03-07;2012-03-07;2012-03-07 07:06:07;2012-03-07 07:06:07;2012-03-07 07:06:07;2012-03-07 07:06:07;[0];[0];[7];[7];[7];[7];[7];[0.7];[2012-03-07,2012-03-07];[2012-03-07 07:06:07];[2012-03-07,2012-03-07];[2012-03-07 07:06:07,2012-03-07 07:06:07,2012-03-07 07:06:07,2012-03-07 07:06:07];[char11,char21,char31];[char11,char21,char31,varchar11,varchar21,varchar31];[char21,char31,varchar11,varchar21,varchar31,string1];[0.7,0.7];ST_Point(12.7, 32.7);ST_Point(12.7, 32.7);98.35620117;36.939093;57.4274406433;57.9020987219;0;7;45.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 34 1, 34 34, 1 34, 1 1)) -7;1;8;8;8;8;8;0.8;0.8;0.8;0.8;0.8;0.8;0.8;0.8;char12;char22;char32;varchar12;varchar22;varchar32;string2;2012-03-08;2012-03-08;2012-03-08 08:07:08;2012-03-08 08:07:08;2012-03-08 08:07:08;2012-03-08 08:07:08;[1];[1];[8];[8];[8];[8];[8];[0.8];[2012-03-08,2012-03-08];[2012-03-08 08:07:08];[2012-03-08,2012-03-08];[2012-03-08 08:07:08,2012-03-08 08:07:08,2012-03-08 08:07:08,2012-03-08 08:07:08];[char12,char22,char32];[char12,char22,char32,varchar12,varchar22,varchar32];[char22,char32,varchar12,varchar22,varchar32,string2];[0.8,0.8];ST_Point(11.1, 31.1);ST_Point(11.1, 31.1);-74.35620117;79.939093;59.4274406433;85.9020987219;0;8;66.321;LINESTRING (1 1, 2 2);POLYGON ((0 0, 38 4, 38 37, 4 1, 0 0)) -8;1;9;9;9;9;9;0.9;0.9;0.9;0.9;0.9;0.9;0.9;0.9;char13;char23;char33;varchar13;varchar23;varchar33;string3;2012-03-09;2012-03-09;2012-03-09 09:08:09;2012-03-09 09:08:09;2012-03-09 09:08:09;2012-03-09 09:08:09;[1];[1];[9];[9];[9];[9];[9];[0.9];[2012-03-09,2012-03-09];[2012-03-09 09:08:09];[2012-03-09,2012-03-09];[2012-03-09 09:08:09,2012-03-09 09:08:09,2012-03-09 09:08:09,2012-03-09 09:08:09];[char13,char23,char33];[char13,char23,char33,varchar13,varchar23,varchar33];[char23,char33,varchar13,varchar23,varchar33,string3];[0.9,0.9];ST_Point(12.1, 32.1);ST_Point(12.1, 32.1);90.35620117;47.939093;89.4274406433;58.9020987219;0;9;88.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 56 0, 67 89, 4 32, 1 1)) -9;1;10;10;10;10;10;1;1;1;1;1;1;1;1;char11;char21;char31;varchar11;varchar21;varchar31;string1;2012-03-10;2012-03-10;2012-03-10 10:09:10;2012-03-10 10:09:10;2012-03-10 10:09:10;2012-03-10 10:09:10;[1];[1];[10];[10];[10];[10];[10];[1];[2012-03-10,2012-03-10];[2012-03-10 10:09:10];[2012-03-10,2012-03-10];[2012-03-10 10:09:10,2012-03-10 10:09:10,2012-03-10 10:09:10,2012-03-10 10:09:10];[char11,char21,char31];[char11,char21,char31,varchar11,varchar21,varchar31];[char21,char31,varchar11,varchar21,varchar31,string1];[1,1];ST_Point(19.1, 39.1);ST_Point(19.1, 39.1);90.35620117;49.939093;89.4274406433;58.9020987219;0;10;76.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 5 0, 10 4, 4 3, 1 1)) -10;1;11;11;11;11;11;1.1;1.1;1.1;1.1;1.1;1.1;1.1;1.1;char12;char22;char32;varchar12;varchar22;varchar32;string2;2012-03-11;2012-03-11;2012-03-11 11:10:11;2012-03-11 11:10:11;2012-03-11 11:10:11;2012-03-11 11:10:11;[1];[1];[11];[11];[11];[11];[11];[1.1];[2012-03-11,2012-03-11];[2012-03-11 11:10:11];[2012-03-11,2012-03-11];[2012-03-11 11:10:11,2012-03-11 11:10:11,2012-03-11 11:10:11,2012-03-11 11:10:11];[char12,char22,char32];[char12,char22,char32,varchar12,varchar22,varchar32];[char22,char32,varchar12,varchar22,varchar32,string2];[1.1,1.1];ST_Point(17.1, 37.1);ST_Point(17.1, 37.1);90.35620117;59.939093;89.4274406433;58.9020987219;0;11;75.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 8 0, 48 34, 4 10, 1 1)) -11;1;12;12;12;12;12;1.2;1.2;1.2;1.2;1.2;1.2;1.2;1.2;char13;char23;char33;varchar13;varchar23;varchar33;string3;2012-03-12;2012-03-12;2012-03-12 12:11:12;2012-03-12 12:11:12;2012-03-12 12:11:12;2012-03-12 12:11:12;[1];[1];[12];[12];[12];[12];[12];[1.2];[2012-03-12,2012-03-12];[2012-03-12 12:11:12];[2012-03-12,2012-03-12];[2012-03-12 12:11:12,2012-03-12 12:11:12,2012-03-12 12:11:12,2012-03-12 12:11:12];[char13,char23,char33];[char13,char23,char33,varchar13,varchar23,varchar33];[char23,char33,varchar13,varchar23,varchar33,string3];[1.2,1.2];ST_Point(22.1, 22.1);ST_Point(22.1, 22.1);90.35620117;39.939093;89.4274406433;58.9020987219;0;12;100.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 9 4, 10 4, 4 1, 1 1)) -null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null \ No newline at end of file +0;0;1;1;1;1;1;0.1;0.1;0.1;0.1;0.1;0.1;0.1;0.1;char11;char21;char31;varchar11;varchar21;varchar31;string1;2012-03-01;2012-03-01;2012-03-01 01:00:01;2012-03-01 01:00:01;2012-03-01 01:00:01;2012-03-01 01:00:01;[0];[0];[1];[1];[1];[1];[1];[0.1];[2012-03-01,2012-03-01];[2012-03-01 01:00:01];[2012-03-01,2012-03-01];[2012-03-01 01:00:01,2012-03-01 01:00:01,2012-03-01 01:00:01,2012-03-01 01:00:01];[char11,char21,char31];[char11,char21,char31,varchar11,varchar21,varchar31];[char21,char31,varchar11,varchar21,varchar31,string1];[0.1,0.1];ST_Point(12.1, 32.1);ST_Point(12.1, 32.1);5;5;116.4274406433;39.9020987219;0;1;4.321;LINESTRING (1 1, 2 2);POLYGON ((0 0, 10 0, 10 10, 0 10, 0 0));{null:1,0:2,0:3};{null:1,1:2,2:3};{null:1,1:2};{null:1,1:2};{null:1,1:2};{null:1,1:2};{null:1,1.1:2};{null:1,1.1:2};{null:1,1.100:2};{null:1,"char1":2};{null:1,"vchar1":2};{null:1,"str1":2};{null:1,2012-03-01:2};{null:1,2012-03-01:2};{1:null,2:0,3:0};{1:null,2:1,3:2};{1:null,2:1};{1:null,2:1};{1:null,2:1};{1:null,2:1};{1:null,2:1.1};{1:null,2:1.1};{1:null,2:1.100};{1:null,2:"char1"};{1:null,2:"vchar1"};{1:null,2:"str1"};{1:null,2:2012-03-01};{1:null,2:2012-03-01} +1;0;2;2;2;2;2;0.2;0.2;0.2;0.2;0.2;0.2;0.2;0.2;char12;char22;char32;varchar12;varchar22;varchar32;string2;2012-03-02;2012-03-02;2012-03-02 02:01:02;2012-03-02 02:01:02;2012-03-02 02:01:02;2012-03-02 02:01:02;[0];[0];[2];[2];[2];[2];[2];[0.2];[2012-03-02,2012-03-02];[2012-03-02 02:01:02];[2012-03-02,2012-03-02];[2012-03-02 02:01:02,2012-03-02 02:01:02,2012-03-02 02:01:02,2012-03-02 02:01:02];[char12,char22,char32];[char12,char22,char32,varchar12,varchar22,varchar32];[char22,char32,varchar12,varchar22,varchar32,string2];[0.2,0.2];ST_Point(12.2, 32.2);ST_Point(12.2, 32.2);126.35620117;-39.939093;56.4274406433;66.9020987219;0;2;5.3;LINESTRING (1 1, 2 2);POLYGON ((1 1, 4 1, 4 4, 1 4, 1 1));{null:1,0:2,0:3};{null:1,2:2,1:3};{null:1,2:2};{null:1,2:2};{null:1,2:2};{null:1,2:2};{null:1,2.2:2};{null:1,2.2:2};{null:1,2.200:2};{null:1,"char2":2};{null:1,"vchar2":2};{null:1,"str2":2};{null:1,2012-03-02:2};{null:1,2012-03-02:2};{1:null,2:0,3:0};{1:null,2:2,3:1};{1:null,2:2};{1:null,2:2};{1:null,2:2};{1:null,2:2};{1:null,2:2.2};{1:null,2:2.2};{1:null,2:2.200};{1:null,2:"char2"};{1:null,2:"vchar2"};{1:null,2:"str2"};{1:null,2:2012-03-02};{1:null,2:2012-03-02} +2;0;3;3;3;3;3;0.3;0.3;0.3;0.3;0.3;0.3;0.3;0.3;char13;char23;char33;varchar13;varchar23;varchar33;string3;2012-03-03;2012-03-03;2012-03-03 03:02:03;2012-03-03 03:02:03;2012-03-03 03:02:03;2012-03-03 03:02:03;[0];[0];[3];[3];[3];[3];[3];[0.3];[2012-03-03,2012-03-03];[2012-03-03 03:02:03];[2012-03-03,2012-03-03];[2012-03-03 03:02:03,2012-03-03 03:02:03,2012-03-03 03:02:03,2012-03-03 03:02:03];[char13,char23,char33];[char13,char23,char33,varchar13,varchar23,varchar33];[char23,char33,varchar13,varchar23,varchar33,string3];[0.3,0.3];ST_Point(12.3, 32.3);ST_Point(12.3, 32.3);16.35620117;19.939093;76.4274406433;46.9020987219;0;3;7.321;LINESTRING (1 1, 2 2);POLYGON ((0 0, 12 0, 10 16, 1 10, 0 0));{null:1,0:2,0:3};{null:1,3:2,0:3};{null:1,3:2};{null:1,3:2};{null:1,3:2};{null:1,3:2};{null:1,3.3:2};{null:1,3.3:2};{null:1,3.300:2};{null:1,"char3":2};{null:1,"vchar3":2};{null:1,"str3":2};{null:1,2012-03-03:2};{null:1,2012-03-03:2};{1:null,2:0,3:0};{1:null,2:3,3:0};{1:null,2:3};{1:null,2:3};{1:null,2:3};{1:null,2:3};{1:null,2:3.3};{1:null,2:3.3};{1:null,2:3.300};{1:null,2:"char3"};{1:null,2:"vchar3"};{1:null,2:"str3"};{1:null,2:2012-03-03};{1:null,2:2012-03-03} +3;0;4;4;4;4;4;0.4;0.4;0.4;0.4;0.4;0.4;0.4;0.4;char11;char21;char31;varchar11;varchar21;varchar31;string1;2012-03-04;2012-03-04;2012-03-04 04:03:04;2012-03-04 04:03:04;2012-03-04 04:03:04;2012-03-04 04:03:04;[0];[0];[4];[4];[4];[4];[4];[0.4];[2012-03-04,2012-03-04];[2012-03-04 04:03:04];[2012-03-04,2012-03-04];[2012-03-04 04:03:04,2012-03-04 04:03:04,2012-03-04 04:03:04,2012-03-04 04:03:04];[char11,char21,char31];[char11,char21,char31,varchar11,varchar21,varchar31];[char21,char31,varchar11,varchar21,varchar31,string1];[0.4,0.4];ST_Point(12.4, 32.4);ST_Point(12.4, 32.4);-46.35620117;39.939093;23.4274406433;-26.9020987219;0;4;3.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 11 4, 42 44, 4 18, 1 1));{null:1,0:2,1:3};{null:1,4:2,9:3};{null:1,4:2};{null:1,4:2};{null:1,4:2};{null:1,4:2};{null:1,4.4:2};{null:1,4.4:2};{null:1,4.400:2};{null:1,"char4":2};{null:1,"vchar4":2};{null:1,"str4":2};{null:1,2012-03-04:2};{null:1,2012-03-04:2};{1:null,2:0,3:1};{1:null,2:4,3:9};{1:null,2:4};{1:null,2:4};{1:null,2:4};{1:null,2:4};{1:null,2:4.4};{1:null,2:4.4};{1:null,2:4.400};{1:null,2:"char4"};{1:null,2:"vchar4"};{1:null,2:"str4"};{1:null,2:2012-03-04};{1:null,2:2012-03-04} +4;0;5;5;5;5;5;0.5;0.5;0.5;0.5;0.5;0.5;0.5;0.5;char12;char22;char32;varchar12;varchar22;varchar32;string2;2012-03-05;2012-03-05;2012-03-05 05:04:05;2012-03-05 05:04:05;2012-03-05 05:04:05;2012-03-05 05:04:05;[0];[0];[5];[5];[5];[5];[5];[0.5];[2012-03-05,2012-03-05];[2012-03-05 05:04:05];[2012-03-05,2012-03-05];[2012-03-05 05:04:05,2012-03-05 05:04:05,2012-03-05 05:04:05,2012-03-05 05:04:05];[char12,char22,char32];[char12,char22,char32,varchar12,varchar22,varchar32];[char22,char32,varchar12,varchar22,varchar32,string2];[0.5,0.5];ST_Point(12.5, 32.5);ST_Point(12.5, 32.5);43.35620117;35.939093;35.4274406433;35.9020987219;0;5;2.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 16 1, 16 16, 1 16, 1 1));{null:1,0:2,1:3};{null:1,5:2,8:3};{null:1,5:2};{null:1,5:2};{null:1,5:2};{null:1,5:2};{null:1,5.5:2};{null:1,5.5:2};{null:1,5.500:2};{null:1,"char5":2};{null:1,"vchar5":2};{null:1,"str5":2};{null:1,2012-03-05:2};{null:1,2012-03-05:2};{1:null,2:0,3:1};{1:null,2:5,3:8};{1:null,2:5};{1:null,2:5};{1:null,2:5};{1:null,2:5};{1:null,2:5.5};{1:null,2:5.5};{1:null,2:5.500};{1:null,2:"char5"};{1:null,2:"vchar5"};{1:null,2:"str5"};{1:null,2:2012-03-05};{1:null,2:2012-03-05} +5;0;6;6;6;6;6;0.6;0.6;0.6;0.6;0.6;0.6;0.6;0.6;char13;char23;char33;varchar13;varchar23;varchar33;string3;2012-03-06;2012-03-06;2012-03-06 06:05:06;2012-03-06 06:05:06;2012-03-06 06:05:06;2012-03-06 06:05:06;[0];[0];[6];[6];[6];[6];[6];[0.6];[2012-03-06,2012-03-06];[2012-03-06 06:05:06];[2012-03-06,2012-03-06];[2012-03-06 06:05:06,2012-03-06 06:05:06,2012-03-06 06:05:06,2012-03-06 06:05:06];[char13,char23,char33];[char13,char23,char33,varchar13,varchar23,varchar33];[char23,char33,varchar13,varchar23,varchar33,string3];[0.6,0.6];ST_Point(12.6, 32.6);ST_Point(12.6, 32.6);47.35620117;26.939093;47.4274406433;75.9020987219;0;6;33.321;LINESTRING (1 1, 2 2);POLYGON ((4 4, 45 4, 45 45, 4 45, 4 4));{null:1,0:2,1:3};{null:1,6:2,7:3};{null:1,6:2};{null:1,6:2};{null:1,6:2};{null:1,6:2};{null:1,6.6:2};{null:1,6.6:2};{null:1,6.600:2};{null:1,"char6":2};{null:1,"vchar6":2};{null:1,"str6":2};{null:1,2012-03-06:2};{null:1,2012-03-06:2};{1:null,2:0,3:1};{1:null,2:6,3:7};{1:null,2:6};{1:null,2:6};{1:null,2:6};{1:null,2:6};{1:null,2:6.6};{1:null,2:6.6};{1:null,2:6.600};{1:null,2:"char6"};{1:null,2:"vchar6"};{1:null,2:"str6"};{1:null,2:2012-03-06};{1:null,2:2012-03-06} +6;0;7;7;7;7;7;0.7;0.7;0.7;0.7;0.7;0.7;0.7;0.7;char11;char21;char31;varchar11;varchar21;varchar31;string1;2012-03-07;2012-03-07;2012-03-07 07:06:07;2012-03-07 07:06:07;2012-03-07 07:06:07;2012-03-07 07:06:07;[0];[0];[7];[7];[7];[7];[7];[0.7];[2012-03-07,2012-03-07];[2012-03-07 07:06:07];[2012-03-07,2012-03-07];[2012-03-07 07:06:07,2012-03-07 07:06:07,2012-03-07 07:06:07,2012-03-07 07:06:07];[char11,char21,char31];[char11,char21,char31,varchar11,varchar21,varchar31];[char21,char31,varchar11,varchar21,varchar31,string1];[0.7,0.7];ST_Point(12.7, 32.7);ST_Point(12.7, 32.7);98.35620117;36.939093;57.4274406433;57.9020987219;0;7;45.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 34 1, 34 34, 1 34, 1 1));{null:1,1:2,0:3};{null:1,7:2,6:3};{null:1,7:2};{null:1,7:2};{null:1,7:2};{null:1,7:2};{null:1,7.7:2};{null:1,7.7:2};{null:1,7.700:2};{null:1,"char7":2};{null:1,"vchar7":2};{null:1,"str7":2};{null:1,2012-03-07:2};{null:1,2012-03-07:2};{1:null,2:1,3:0};{1:null,2:7,3:6};{1:null,2:7};{1:null,2:7};{1:null,2:7};{1:null,2:7};{1:null,2:7.7};{1:null,2:7.7};{1:null,2:7.700};{1:null,2:"char7"};{1:null,2:"vchar7"};{1:null,2:"str7"};{1:null,2:2012-03-07};{1:null,2:2012-03-07} +7;1;8;8;8;8;8;0.8;0.8;0.8;0.8;0.8;0.8;0.8;0.8;char12;char22;char32;varchar12;varchar22;varchar32;string2;2012-03-08;2012-03-08;2012-03-08 08:07:08;2012-03-08 08:07:08;2012-03-08 08:07:08;2012-03-08 08:07:08;[1];[1];[8];[8];[8];[8];[8];[0.8];[2012-03-08,2012-03-08];[2012-03-08 08:07:08];[2012-03-08,2012-03-08];[2012-03-08 08:07:08,2012-03-08 08:07:08,2012-03-08 08:07:08,2012-03-08 08:07:08];[char12,char22,char32];[char12,char22,char32,varchar12,varchar22,varchar32];[char22,char32,varchar12,varchar22,varchar32,string2];[0.8,0.8];ST_Point(11.1, 31.1);ST_Point(11.1, 31.1);-74.35620117;79.939093;59.4274406433;85.9020987219;0;8;66.321;LINESTRING (1 1, 2 2);POLYGON ((0 0, 38 4, 38 37, 4 1, 0 0));{null:1,1:2,0:3};{null:1,8:2,5:3};{null:1,8:2};{null:1,8:2};{null:1,8:2};{null:1,8:2};{null:1,8.8:2};{null:1,8.8:2};{null:1,8.800:2};{null:1,"char8":2};{null:1,"vchar8":2};{null:1,"str8":2};{null:1,2012-03-08:2};{null:1,2012-03-08:2};{1:null,2:1,3:0};{1:null,2:8,3:5};{1:null,2:8};{1:null,2:8};{1:null,2:8};{1:null,2:8};{1:null,2:8.8};{1:null,2:8.8};{1:null,2:8.800};{1:null,2:"char8"};{1:null,2:"vchar8"};{1:null,2:"str8"};{1:null,2:2012-03-08};{1:null,2:2012-03-08} +8;1;9;9;9;9;9;0.9;0.9;0.9;0.9;0.9;0.9;0.9;0.9;char13;char23;char33;varchar13;varchar23;varchar33;string3;2012-03-09;2012-03-09;2012-03-09 09:08:09;2012-03-09 09:08:09;2012-03-09 09:08:09;2012-03-09 09:08:09;[1];[1];[9];[9];[9];[9];[9];[0.9];[2012-03-09,2012-03-09];[2012-03-09 09:08:09];[2012-03-09,2012-03-09];[2012-03-09 09:08:09,2012-03-09 09:08:09,2012-03-09 09:08:09,2012-03-09 09:08:09];[char13,char23,char33];[char13,char23,char33,varchar13,varchar23,varchar33];[char23,char33,varchar13,varchar23,varchar33,string3];[0.9,0.9];ST_Point(12.1, 32.1);ST_Point(12.1, 32.1);90.35620117;47.939093;89.4274406433;58.9020987219;0;9;88.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 56 0, 67 89, 4 32, 1 1));{null:1,1:2,0:3};{null:1,9:2,4:3};{null:1,9:2};{null:1,9:2};{null:1,9:2};{null:1,9:2};{null:1,9.9:2};{null:1,9.9:2};{null:1,9.900:2};{null:1,"char9":2};{null:1,"vchar9":2};{null:1,"str9":2};{null:1,2012-03-09:2};{null:1,2012-03-09:2};{1:null,2:1,3:0};{1:null,2:9,3:4};{1:null,2:9};{1:null,2:9};{1:null,2:9};{1:null,2:9};{1:null,2:9.9};{1:null,2:9.9};{1:null,2:9.900};{1:null,2:"char9"};{1:null,2:"vchar9"};{1:null,2:"str9"};{1:null,2:2012-03-09};{1:null,2:2012-03-09} +9;1;10;10;10;10;10;1;1;1;1;1;1;1;1;char11;char21;char31;varchar11;varchar21;varchar31;string1;2012-03-10;2012-03-10;2012-03-10 10:09:10;2012-03-10 10:09:10;2012-03-10 10:09:10;2012-03-10 10:09:10;[1];[1];[10];[10];[10];[10];[10];[1];[2012-03-10,2012-03-10];[2012-03-10 10:09:10];[2012-03-10,2012-03-10];[2012-03-10 10:09:10,2012-03-10 10:09:10,2012-03-10 10:09:10,2012-03-10 10:09:10];[char11,char21,char31];[char11,char21,char31,varchar11,varchar21,varchar31];[char21,char31,varchar11,varchar21,varchar31,string1];[1,1];ST_Point(19.1, 39.1);ST_Point(19.1, 39.1);90.35620117;49.939093;89.4274406433;58.9020987219;0;10;76.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 5 0, 10 4, 4 3, 1 1));{null:1,1:2,1:3};{null:1,0:2,3:3};{null:1,0:2};{null:1,0:2};{null:1,0:2};{null:1,0:2};{null:1,0.0:2};{null:1,0.0:2};{null:1,0.000:2};{null:1,"char0":2};{null:1,"vchar0":2};{null:1,"str0":2};{null:1,2012-03-10:2};{null:1,2012-03-10:2};{1:null,2:1,3:1};{1:null,2:0,3:3};{1:null,2:0};{1:null,2:0};{1:null,2:0};{1:null,2:0};{1:null,2:0.0};{1:null,2:0.0};{1:null,2:0.000};{1:null,2:"char0"};{1:null,2:"vchar0"};{1:null,2:"str0"};{1:null,2:2012-03-10};{1:null,2:2012-03-10} +10;1;11;11;11;11;11;1.1;1.1;1.1;1.1;1.1;1.1;1.1;1.1;char12;char22;char32;varchar12;varchar22;varchar32;string2;2012-03-11;2012-03-11;2012-03-11 11:10:11;2012-03-11 11:10:11;2012-03-11 11:10:11;2012-03-11 11:10:11;[1];[1];[11];[11];[11];[11];[11];[1.1];[2012-03-11,2012-03-11];[2012-03-11 11:10:11];[2012-03-11,2012-03-11];[2012-03-11 11:10:11,2012-03-11 11:10:11,2012-03-11 11:10:11,2012-03-11 11:10:11];[char12,char22,char32];[char12,char22,char32,varchar12,varchar22,varchar32];[char22,char32,varchar12,varchar22,varchar32,string2];[1.1,1.1];ST_Point(17.1, 37.1);ST_Point(17.1, 37.1);90.35620117;59.939093;89.4274406433;58.9020987219;0;11;75.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 8 0, 48 34, 4 10, 1 1));{null:1,1:2,1:3};{null:1,1:2,2:3};{null:1,1:2};{null:1,1:2};{null:1,1:2};{null:1,1:2};{null:1,1.1:2};{null:1,1.1:2};{null:1,1.100:2};{null:1,"char1":2};{null:1,"vchar1":2};{null:1,"str1":2};{null:1,2012-03-11:2};{null:1,2012-03-11:2};{1:null,2:1,3:1};{1:null,2:1,3:2};{1:null,2:1};{1:null,2:1};{1:null,2:1};{1:null,2:1};{1:null,2:1.1};{1:null,2:1.1};{1:null,2:1.100};{1:null,2:"char1"};{1:null,2:"vchar1"};{1:null,2:"str1"};{1:null,2:2012-03-11};{1:null,2:2012-03-11} +11;1;12;12;12;12;12;1.2;1.2;1.2;1.2;1.2;1.2;1.2;1.2;char13;char23;char33;varchar13;varchar23;varchar33;string3;2012-03-12;2012-03-12;2012-03-12 12:11:12;2012-03-12 12:11:12;2012-03-12 12:11:12;2012-03-12 12:11:12;[1];[1];[12];[12];[12];[12];[12];[1.2];[2012-03-12,2012-03-12];[2012-03-12 12:11:12];[2012-03-12,2012-03-12];[2012-03-12 12:11:12,2012-03-12 12:11:12,2012-03-12 12:11:12,2012-03-12 12:11:12];[char13,char23,char33];[char13,char23,char33,varchar13,varchar23,varchar33];[char23,char33,varchar13,varchar23,varchar33,string3];[1.2,1.2];ST_Point(22.1, 22.1);ST_Point(22.1, 22.1);90.35620117;39.939093;89.4274406433;58.9020987219;0;12;100.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 9 4, 10 4, 4 1, 1 1));{null:1,1:2,1:3};{null:1,2:2,1:3};{null:1,2:2};{null:1,2:2};{null:1,2:2};{null:1,2:2};{null:1,2.2:2};{null:1,2.2:2};{null:1,2.200:2};{null:1,"char2":2};{null:1,"vchar2":2};{null:1,"str2":2};{null:1,2012-03-12:2};{null:1,2012-03-12:2};{1:null,2:1,3:1};{1:null,2:2,3:1};{1:null,2:2};{1:null,2:2};{1:null,2:2};{1:null,2:2};{1:null,2:2.2};{1:null,2:2.2};{1:null,2:2.200};{1:null,2:"char2"};{1:null,2:"vchar2"};{1:null,2:"str2"};{1:null,2:2012-03-12};{1:null,2:2012-03-12} +null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null \ No newline at end of file diff --git a/regression-test/data/nereids_function_p0/scalar_function/C.out b/regression-test/data/nereids_function_p0/scalar_function/C.out index 7f60e9d43fd45a9..17aae4973047a50 100644 --- a/regression-test/data/nereids_function_p0/scalar_function/C.out +++ b/regression-test/data/nereids_function_p0/scalar_function/C.out @@ -1101,3 +1101,9 @@ varchar13 0.4535961214255773 0.3623577544766736 +-- !char -- +Doris Doris + +-- !convert -- +1 1 + diff --git a/regression-test/data/nereids_function_p0/scalar_function/E.out b/regression-test/data/nereids_function_p0/scalar_function/E.out index ccaa576f128d6fc..f543bd22d9be7d6 100644 --- a/regression-test/data/nereids_function_p0/scalar_function/E.out +++ b/regression-test/data/nereids_function_p0/scalar_function/E.out @@ -173,3 +173,6 @@ true +-- !sql_encryptkey -- +ABCD123456789 ABCD123456789 + diff --git a/regression-test/data/nereids_function_p0/scalar_function/Map.out b/regression-test/data/nereids_function_p0/scalar_function/Map.out new file mode 100644 index 000000000000000..ca4185da385ceda --- /dev/null +++ b/regression-test/data/nereids_function_p0/scalar_function/Map.out @@ -0,0 +1,2843 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !element_at_bool_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +3 + +-- !element_at_tint_tint -- +\N +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_sint_tint -- +\N +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_int_tint -- +\N +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_bint_tint -- +\N +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_lint_tint -- +\N +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_float_tint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +2 + +-- !element_at_dbl_tint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +2 + +-- !element_at_dcml_tint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +2 + +-- !element_at_chr_tint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + +-- !element_at_vchr_tint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + +-- !element_at_str_tint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + +-- !element_at_date_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_dtm_tint -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + +-- !element_at_bool_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +3 + +-- !element_at_tint_tint_notnull -- +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_sint_tint_notnull -- +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_int_tint_notnull -- +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_bint_tint_notnull -- +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_lint_tint_notnull -- +\N +\N +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_float_tint_notnull -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +2 + +-- !element_at_dbl_tint_notnull -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +2 + +-- !element_at_dcml_tint_notnull -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +2 + +-- !element_at_chr_tint_notnull -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + +-- !element_at_vchr_tint_notnull -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + +-- !element_at_str_tint_notnull -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + +-- !element_at_date_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !element_at_dtm_tint_notnull -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + +-- !map_size_bool_tint -- +\N +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 + +-- !map_size_tint_tint -- +\N +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 + +-- !map_size_sint_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_int_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_bint_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_lint_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_float_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_dbl_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_dcml_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_chr_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_vchr_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_str_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_date_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_dtm_tint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_bool -- +\N +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 + +-- !map_size_int_int -- +\N +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 + +-- !map_size_tint_sint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_int -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_bint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_lint -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_float -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_dbl -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_dcml -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_chr -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_vchr -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_str -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_date -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_dtm -- +\N +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_bool_tint_notnull -- +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 + +-- !map_size_tint_tint_notnull -- +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 + +-- !map_size_sint_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_int_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_bint_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_lint_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_float_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_dbl_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_dcml_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_chr_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_vchr_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_str_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_date_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_dtm_tint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_bool_notnull -- +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 + +-- !map_size_int_int_notnull -- +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 +3 + +-- !map_size_tint_sint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_int_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_bint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_lint_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_float_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_dbl_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_dcml_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_chr_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_vchr_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_str_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_date_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_size_tint_dtm_notnull -- +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !map_keys_bool_tint -- +\N +[NULL, 0, 0] +[NULL, 0, 0] +[NULL, 0, 0] +[NULL, 0, 1] +[NULL, 0, 1] +[NULL, 0, 1] +[NULL, 1, 0] +[NULL, 1, 0] +[NULL, 1, 0] +[NULL, 1, 1] +[NULL, 1, 1] +[NULL, 1, 1] + +-- !map_keys_tint_tint -- +\N +[NULL, 0, 3] +[NULL, 1, 2] +[NULL, 1, 2] +[NULL, 2, 1] +[NULL, 2, 1] +[NULL, 3, 0] +[NULL, 4, 9] +[NULL, 5, 8] +[NULL, 6, 7] +[NULL, 7, 6] +[NULL, 8, 5] +[NULL, 9, 4] + +-- !map_keys_sint_tint -- +\N +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_keys_int_tint -- +\N +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_keys_bint_tint -- +\N +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_keys_lint_tint -- +\N +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_keys_float_tint -- +\N +[NULL, 0] +[NULL, 1.1] +[NULL, 1.1] +[NULL, 2.2] +[NULL, 2.2] +[NULL, 3.3] +[NULL, 4.4] +[NULL, 5.5] +[NULL, 6.6] +[NULL, 7.7] +[NULL, 8.8] +[NULL, 9.9] + +-- !map_keys_dbl_tint -- +\N +[NULL, 0] +[NULL, 1.1] +[NULL, 1.1] +[NULL, 2.2] +[NULL, 2.2] +[NULL, 3.3] +[NULL, 4.4] +[NULL, 5.5] +[NULL, 6.6] +[NULL, 7.7] +[NULL, 8.8] +[NULL, 9.9] + +-- !map_keys_dcml_tint -- +\N +[NULL, 0.000000000] +[NULL, 1.100000000] +[NULL, 1.100000000] +[NULL, 2.200000000] +[NULL, 2.200000000] +[NULL, 3.300000000] +[NULL, 4.400000000] +[NULL, 5.500000000] +[NULL, 6.600000000] +[NULL, 7.700000000] +[NULL, 8.800000000] +[NULL, 9.900000000] + +-- !map_keys_chr_tint -- +\N +[NULL, "char0"] +[NULL, "char1"] +[NULL, "char1"] +[NULL, "char2"] +[NULL, "char2"] +[NULL, "char3"] +[NULL, "char4"] +[NULL, "char5"] +[NULL, "char6"] +[NULL, "char7"] +[NULL, "char8"] +[NULL, "char9"] + +-- !map_keys_vchr_tint -- +\N +[NULL, "vchar0"] +[NULL, "vchar1"] +[NULL, "vchar1"] +[NULL, "vchar2"] +[NULL, "vchar2"] +[NULL, "vchar3"] +[NULL, "vchar4"] +[NULL, "vchar5"] +[NULL, "vchar6"] +[NULL, "vchar7"] +[NULL, "vchar8"] +[NULL, "vchar9"] + +-- !map_keys_str_tint -- +\N +[NULL, "str0"] +[NULL, "str1"] +[NULL, "str1"] +[NULL, "str2"] +[NULL, "str2"] +[NULL, "str3"] +[NULL, "str4"] +[NULL, "str5"] +[NULL, "str6"] +[NULL, "str7"] +[NULL, "str8"] +[NULL, "str9"] + +-- !map_keys_date_tint -- +\N +[NULL, 2012-03-01] +[NULL, 2012-03-02] +[NULL, 2012-03-03] +[NULL, 2012-03-04] +[NULL, 2012-03-05] +[NULL, 2012-03-06] +[NULL, 2012-03-07] +[NULL, 2012-03-08] +[NULL, 2012-03-09] +[NULL, 2012-03-10] +[NULL, 2012-03-11] +[NULL, 2012-03-12] + +-- !map_keys_dtm_tint -- +\N +[NULL, 2012-03-01 00:00:00] +[NULL, 2012-03-02 00:00:00] +[NULL, 2012-03-03 00:00:00] +[NULL, 2012-03-04 00:00:00] +[NULL, 2012-03-05 00:00:00] +[NULL, 2012-03-06 00:00:00] +[NULL, 2012-03-07 00:00:00] +[NULL, 2012-03-08 00:00:00] +[NULL, 2012-03-09 00:00:00] +[NULL, 2012-03-10 00:00:00] +[NULL, 2012-03-11 00:00:00] +[NULL, 2012-03-12 00:00:00] + +-- !map_keys_bool_tint_notnull -- +[NULL, 0, 0] +[NULL, 0, 0] +[NULL, 0, 0] +[NULL, 0, 1] +[NULL, 0, 1] +[NULL, 0, 1] +[NULL, 1, 0] +[NULL, 1, 0] +[NULL, 1, 0] +[NULL, 1, 1] +[NULL, 1, 1] +[NULL, 1, 1] + +-- !map_keys_tint_tint_notnull -- +[NULL, 0, 3] +[NULL, 1, 2] +[NULL, 1, 2] +[NULL, 2, 1] +[NULL, 2, 1] +[NULL, 3, 0] +[NULL, 4, 9] +[NULL, 5, 8] +[NULL, 6, 7] +[NULL, 7, 6] +[NULL, 8, 5] +[NULL, 9, 4] + +-- !map_keys_sint_tint_notnull -- +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_keys_int_tint_notnull -- +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_keys_bint_tint_notnull -- +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_keys_lint_tint_notnull -- +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_keys_float_tint_notnull -- +[NULL, 0] +[NULL, 1.1] +[NULL, 1.1] +[NULL, 2.2] +[NULL, 2.2] +[NULL, 3.3] +[NULL, 4.4] +[NULL, 5.5] +[NULL, 6.6] +[NULL, 7.7] +[NULL, 8.8] +[NULL, 9.9] + +-- !map_keys_dbl_tint_notnull -- +[NULL, 0] +[NULL, 1.1] +[NULL, 1.1] +[NULL, 2.2] +[NULL, 2.2] +[NULL, 3.3] +[NULL, 4.4] +[NULL, 5.5] +[NULL, 6.6] +[NULL, 7.7] +[NULL, 8.8] +[NULL, 9.9] + +-- !map_keys_dcml_tint_notnull -- +[NULL, 0.000000000] +[NULL, 1.100000000] +[NULL, 1.100000000] +[NULL, 2.200000000] +[NULL, 2.200000000] +[NULL, 3.300000000] +[NULL, 4.400000000] +[NULL, 5.500000000] +[NULL, 6.600000000] +[NULL, 7.700000000] +[NULL, 8.800000000] +[NULL, 9.900000000] + +-- !map_keys_chr_tint_notnull -- +[NULL, "char0"] +[NULL, "char1"] +[NULL, "char1"] +[NULL, "char2"] +[NULL, "char2"] +[NULL, "char3"] +[NULL, "char4"] +[NULL, "char5"] +[NULL, "char6"] +[NULL, "char7"] +[NULL, "char8"] +[NULL, "char9"] + +-- !map_keys_vchr_tint_notnull -- +[NULL, "vchar0"] +[NULL, "vchar1"] +[NULL, "vchar1"] +[NULL, "vchar2"] +[NULL, "vchar2"] +[NULL, "vchar3"] +[NULL, "vchar4"] +[NULL, "vchar5"] +[NULL, "vchar6"] +[NULL, "vchar7"] +[NULL, "vchar8"] +[NULL, "vchar9"] + +-- !map_keys_str_tint_notnull -- +[NULL, "str0"] +[NULL, "str1"] +[NULL, "str1"] +[NULL, "str2"] +[NULL, "str2"] +[NULL, "str3"] +[NULL, "str4"] +[NULL, "str5"] +[NULL, "str6"] +[NULL, "str7"] +[NULL, "str8"] +[NULL, "str9"] + +-- !map_keys_date_tint_notnull -- +[NULL, 2012-03-01] +[NULL, 2012-03-02] +[NULL, 2012-03-03] +[NULL, 2012-03-04] +[NULL, 2012-03-05] +[NULL, 2012-03-06] +[NULL, 2012-03-07] +[NULL, 2012-03-08] +[NULL, 2012-03-09] +[NULL, 2012-03-10] +[NULL, 2012-03-11] +[NULL, 2012-03-12] + +-- !map_keys_dtm_tint_notnull -- +[NULL, 2012-03-01 00:00:00] +[NULL, 2012-03-02 00:00:00] +[NULL, 2012-03-03 00:00:00] +[NULL, 2012-03-04 00:00:00] +[NULL, 2012-03-05 00:00:00] +[NULL, 2012-03-06 00:00:00] +[NULL, 2012-03-07 00:00:00] +[NULL, 2012-03-08 00:00:00] +[NULL, 2012-03-09 00:00:00] +[NULL, 2012-03-10 00:00:00] +[NULL, 2012-03-11 00:00:00] +[NULL, 2012-03-12 00:00:00] + +-- !map_values_tint_bool -- +\N +[NULL, 0, 0] +[NULL, 0, 0] +[NULL, 0, 0] +[NULL, 0, 1] +[NULL, 0, 1] +[NULL, 0, 1] +[NULL, 1, 0] +[NULL, 1, 0] +[NULL, 1, 0] +[NULL, 1, 1] +[NULL, 1, 1] +[NULL, 1, 1] + +-- !map_values_int_int -- +\N +[NULL, 0, 3] +[NULL, 1, 2] +[NULL, 1, 2] +[NULL, 2, 1] +[NULL, 2, 1] +[NULL, 3, 0] +[NULL, 4, 9] +[NULL, 5, 8] +[NULL, 6, 7] +[NULL, 7, 6] +[NULL, 8, 5] +[NULL, 9, 4] + +-- !map_values_tint_sint -- +\N +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_values_tint_int -- +\N +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_values_tint_bint -- +\N +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_values_tint_lint -- +\N +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_values_tint_float -- +\N +[NULL, 0] +[NULL, 1.1] +[NULL, 1.1] +[NULL, 2.2] +[NULL, 2.2] +[NULL, 3.3] +[NULL, 4.4] +[NULL, 5.5] +[NULL, 6.6] +[NULL, 7.7] +[NULL, 8.8] +[NULL, 9.9] + +-- !map_values_tint_dbl -- +\N +[NULL, 0] +[NULL, 1.1] +[NULL, 1.1] +[NULL, 2.2] +[NULL, 2.2] +[NULL, 3.3] +[NULL, 4.4] +[NULL, 5.5] +[NULL, 6.6] +[NULL, 7.7] +[NULL, 8.8] +[NULL, 9.9] + +-- !map_values_tint_dcml -- +\N +[NULL, 0.000000000] +[NULL, 1.100000000] +[NULL, 1.100000000] +[NULL, 2.200000000] +[NULL, 2.200000000] +[NULL, 3.300000000] +[NULL, 4.400000000] +[NULL, 5.500000000] +[NULL, 6.600000000] +[NULL, 7.700000000] +[NULL, 8.800000000] +[NULL, 9.900000000] + +-- !map_values_tint_chr -- +\N +[NULL, "char0"] +[NULL, "char1"] +[NULL, "char1"] +[NULL, "char2"] +[NULL, "char2"] +[NULL, "char3"] +[NULL, "char4"] +[NULL, "char5"] +[NULL, "char6"] +[NULL, "char7"] +[NULL, "char8"] +[NULL, "char9"] + +-- !map_values_tint_vchr -- +\N +[NULL, "vchar0"] +[NULL, "vchar1"] +[NULL, "vchar1"] +[NULL, "vchar2"] +[NULL, "vchar2"] +[NULL, "vchar3"] +[NULL, "vchar4"] +[NULL, "vchar5"] +[NULL, "vchar6"] +[NULL, "vchar7"] +[NULL, "vchar8"] +[NULL, "vchar9"] + +-- !map_values_tint_str -- +\N +[NULL, "str0"] +[NULL, "str1"] +[NULL, "str1"] +[NULL, "str2"] +[NULL, "str2"] +[NULL, "str3"] +[NULL, "str4"] +[NULL, "str5"] +[NULL, "str6"] +[NULL, "str7"] +[NULL, "str8"] +[NULL, "str9"] + +-- !map_values_tint_date -- +\N +[NULL, 2012-03-01] +[NULL, 2012-03-02] +[NULL, 2012-03-03] +[NULL, 2012-03-04] +[NULL, 2012-03-05] +[NULL, 2012-03-06] +[NULL, 2012-03-07] +[NULL, 2012-03-08] +[NULL, 2012-03-09] +[NULL, 2012-03-10] +[NULL, 2012-03-11] +[NULL, 2012-03-12] + +-- !map_values_tint_dtm -- +\N +[NULL, 2012-03-01 00:00:00] +[NULL, 2012-03-02 00:00:00] +[NULL, 2012-03-03 00:00:00] +[NULL, 2012-03-04 00:00:00] +[NULL, 2012-03-05 00:00:00] +[NULL, 2012-03-06 00:00:00] +[NULL, 2012-03-07 00:00:00] +[NULL, 2012-03-08 00:00:00] +[NULL, 2012-03-09 00:00:00] +[NULL, 2012-03-10 00:00:00] +[NULL, 2012-03-11 00:00:00] +[NULL, 2012-03-12 00:00:00] + +-- !map_values_tint_bool_notnull -- +[NULL, 0, 0] +[NULL, 0, 0] +[NULL, 0, 0] +[NULL, 0, 1] +[NULL, 0, 1] +[NULL, 0, 1] +[NULL, 1, 0] +[NULL, 1, 0] +[NULL, 1, 0] +[NULL, 1, 1] +[NULL, 1, 1] +[NULL, 1, 1] + +-- !map_values_int_int_notnull -- +[NULL, 0, 3] +[NULL, 1, 2] +[NULL, 1, 2] +[NULL, 2, 1] +[NULL, 2, 1] +[NULL, 3, 0] +[NULL, 4, 9] +[NULL, 5, 8] +[NULL, 6, 7] +[NULL, 7, 6] +[NULL, 8, 5] +[NULL, 9, 4] + +-- !map_values_tint_sint_notnull -- +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_values_tint_int_notnull -- +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_values_tint_bint_notnull -- +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_values_tint_lint_notnull -- +[NULL, 0] +[NULL, 1] +[NULL, 1] +[NULL, 2] +[NULL, 2] +[NULL, 3] +[NULL, 4] +[NULL, 5] +[NULL, 6] +[NULL, 7] +[NULL, 8] +[NULL, 9] + +-- !map_values_tint_float_notnull -- +[NULL, 0] +[NULL, 1.1] +[NULL, 1.1] +[NULL, 2.2] +[NULL, 2.2] +[NULL, 3.3] +[NULL, 4.4] +[NULL, 5.5] +[NULL, 6.6] +[NULL, 7.7] +[NULL, 8.8] +[NULL, 9.9] + +-- !map_values_tint_dbl_notnull -- +[NULL, 0] +[NULL, 1.1] +[NULL, 1.1] +[NULL, 2.2] +[NULL, 2.2] +[NULL, 3.3] +[NULL, 4.4] +[NULL, 5.5] +[NULL, 6.6] +[NULL, 7.7] +[NULL, 8.8] +[NULL, 9.9] + +-- !map_values_tint_dcml_notnull -- +[NULL, 0.000000000] +[NULL, 1.100000000] +[NULL, 1.100000000] +[NULL, 2.200000000] +[NULL, 2.200000000] +[NULL, 3.300000000] +[NULL, 4.400000000] +[NULL, 5.500000000] +[NULL, 6.600000000] +[NULL, 7.700000000] +[NULL, 8.800000000] +[NULL, 9.900000000] + +-- !map_values_tint_chr_notnull -- +[NULL, "char0"] +[NULL, "char1"] +[NULL, "char1"] +[NULL, "char2"] +[NULL, "char2"] +[NULL, "char3"] +[NULL, "char4"] +[NULL, "char5"] +[NULL, "char6"] +[NULL, "char7"] +[NULL, "char8"] +[NULL, "char9"] + +-- !map_values_tint_vchr_notnull -- +[NULL, "vchar0"] +[NULL, "vchar1"] +[NULL, "vchar1"] +[NULL, "vchar2"] +[NULL, "vchar2"] +[NULL, "vchar3"] +[NULL, "vchar4"] +[NULL, "vchar5"] +[NULL, "vchar6"] +[NULL, "vchar7"] +[NULL, "vchar8"] +[NULL, "vchar9"] + +-- !map_values_tint_str_notnull -- +[NULL, "str0"] +[NULL, "str1"] +[NULL, "str1"] +[NULL, "str2"] +[NULL, "str2"] +[NULL, "str3"] +[NULL, "str4"] +[NULL, "str5"] +[NULL, "str6"] +[NULL, "str7"] +[NULL, "str8"] +[NULL, "str9"] + +-- !map_values_tint_date_notnull -- +[NULL, 2012-03-01] +[NULL, 2012-03-02] +[NULL, 2012-03-03] +[NULL, 2012-03-04] +[NULL, 2012-03-05] +[NULL, 2012-03-06] +[NULL, 2012-03-07] +[NULL, 2012-03-08] +[NULL, 2012-03-09] +[NULL, 2012-03-10] +[NULL, 2012-03-11] +[NULL, 2012-03-12] + +-- !map_values_tint_dtm_notnull -- +[NULL, 2012-03-01 00:00:00] +[NULL, 2012-03-02 00:00:00] +[NULL, 2012-03-03 00:00:00] +[NULL, 2012-03-04 00:00:00] +[NULL, 2012-03-05 00:00:00] +[NULL, 2012-03-06 00:00:00] +[NULL, 2012-03-07 00:00:00] +[NULL, 2012-03-08 00:00:00] +[NULL, 2012-03-09 00:00:00] +[NULL, 2012-03-10 00:00:00] +[NULL, 2012-03-11 00:00:00] +[NULL, 2012-03-12 00:00:00] + +-- !map_contains_key_bool_tint -- +\N +true +true +true +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_tint_tint -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_sint_tint -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_int_tint -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_bint_tint -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_lint_tint -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_float_tint -- +\N +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_key_dbl_tint -- +\N +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_key_dcml_tint -- +\N +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_key_chr_tint -- +\N +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_key_vchr_tint -- +\N +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_key_str_tint -- +\N +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_key_date_tint -- +\N +true +true +true +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_dtm_tint -- +\N +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_key_bool_tint_notnull -- +true +true +true +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_tint_tint_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_sint_tint_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_int_tint_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_bint_tint_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_lint_tint_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_float_tint_notnull -- +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_key_dbl_tint_notnull -- +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_key_dcml_tint_notnull -- +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_key_chr_tint_notnull -- +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_key_vchr_tint_notnull -- +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_key_str_tint_notnull -- +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_key_date_tint_notnull -- +true +true +true +true +true +true +true +true +true +true +true +true + +-- !map_contains_key_dtm_tint_notnull -- +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_value_tint_bool -- +\N +true +true +true +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_int_int -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_sint -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_int -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_bint -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_lint -- +\N +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_float -- +\N +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_value_tint_dbl -- +\N +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_value_tint_dcml -- +\N +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_value_tint_chr -- +\N +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_value_tint_vchr -- +\N +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_value_tint_str -- +\N +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_value_tint_date -- +\N +true +true +true +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_dtm -- +\N +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_value_tint_bool_notnull -- +true +true +true +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_int_int_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_sint_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_int_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_bint_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_lint_notnull -- +false +false +false +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_float_notnull -- +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_value_tint_dbl_notnull -- +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_value_tint_dcml_notnull -- +false +false +false +false +false +false +false +false +false +false +false +true + +-- !map_contains_value_tint_chr_notnull -- +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_value_tint_vchr_notnull -- +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_value_tint_str_notnull -- +false +false +false +false +false +false +false +false +false +false +false +false + +-- !map_contains_value_tint_date_notnull -- +true +true +true +true +true +true +true +true +true +true +true +true + +-- !map_contains_value_tint_dtm_notnull -- +false +false +false +false +false +false +false +false +false +false +false +false + diff --git a/regression-test/data/nereids_p0/delete/delete_cte.out b/regression-test/data/nereids_p0/delete/delete_cte.out new file mode 100644 index 000000000000000..ac92b46e48de875 --- /dev/null +++ b/regression-test/data/nereids_p0/delete/delete_cte.out @@ -0,0 +1,15 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 \N 2 1 1.0 \N +1 10 1 1 1.0 2000-01-01 +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + +-- !sql -- +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + diff --git a/regression-test/data/nereids_p0/hint/test_leading.out b/regression-test/data/nereids_p0/hint/test_leading.out new file mode 100644 index 000000000000000..5eaf8a864c4fd1f --- /dev/null +++ b/regression-test/data/nereids_p0/hint/test_leading.out @@ -0,0 +1,430 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select1 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +--------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t1] + +Used: leading(t2 t1) +UnUsed: +SyntaxError: + +-- !select2 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------PhysicalOlapScan[t2] + +Used: leading(t1 t2) +UnUsed: +SyntaxError: + +-- !select3 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +--------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +----------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t3] + +Used: leading(t1 t2 t3) +UnUsed: +SyntaxError: + +-- !select4 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +------------PhysicalOlapScan[t2] +------------PhysicalDistribute +--------------PhysicalOlapScan[t3] + +Used: leading(t1 { t2 t3 }) +UnUsed: +SyntaxError: + +-- !select5 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t3.c3 = t4.c4) +--------PhysicalDistribute +----------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +------------PhysicalOlapScan[t1] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +----------------PhysicalOlapScan[t2] +----------------PhysicalDistribute +------------------PhysicalOlapScan[t3] +--------PhysicalDistribute +----------PhysicalOlapScan[t4] + +Used: leading(t1 { t2 t3 } t4) +UnUsed: +SyntaxError: + +-- !select6 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +--------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +----------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------hashJoin[INNER_JOIN](t3.c3 = t4.c4) +------------PhysicalOlapScan[t3] +------------PhysicalDistribute +--------------PhysicalOlapScan[t4] + +Used: leading({ t1 t2 } { t3 t4 }) +UnUsed: +SyntaxError: + +-- !select7 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +--------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +----------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------hashJoin[INNER_JOIN](t3.c3 = t4.c4) +------------PhysicalOlapScan[t3] +------------PhysicalDistribute +--------------PhysicalOlapScan[t4] + +Used: leading({ t1 t2 } { t3 t4 }) +UnUsed: +SyntaxError: + +-- !select8 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t3.c3) +--------hashJoin[LEFT_OUTER_JOIN](t1.c1 = t2.c2) +----------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t3] + +-- !select9 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[LEFT_OUTER_JOIN](t1.c1 = t2.c2) +--------hashJoin[INNER_JOIN](t1.c1 = t3.c3) +----------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalOlapScan[t3] +--------PhysicalDistribute +----------PhysicalOlapScan[t2] + +Used: leading(t1 t3 t2) +UnUsed: +SyntaxError: + +-- !select10 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[LEFT_OUTER_JOIN](t1.c1 = t3.c3) +--------hashJoin[LEFT_OUTER_JOIN](t1.c1 = t2.c2) +----------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t3] + +-- !select11 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[LEFT_OUTER_JOIN](t1.c1 = t2.c2) +--------hashJoin[LEFT_OUTER_JOIN](t1.c1 = t3.c3) +----------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalOlapScan[t3] +--------PhysicalDistribute +----------PhysicalOlapScan[t2] + +Used: leading(t1 t3 t2) +UnUsed: +SyntaxError: + +-- !select12 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[LEFT_OUTER_JOIN](t2.c2 = t3.c3) +--------PhysicalDistribute +----------hashJoin[LEFT_OUTER_JOIN](t1.c1 = t2.c2) +------------PhysicalOlapScan[t1] +------------PhysicalDistribute +--------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t3] + +Used: +UnUsed: leading(t1 { t2 t3 }) +SyntaxError: + +-- !select13 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +------------PhysicalOlapScan[t2] +------------PhysicalDistribute +--------------PhysicalOlapScan[t3] + +Used: leading(t1 { t2 t3 }) +UnUsed: +SyntaxError: + +-- !select14 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[LEFT_OUTER_JOIN](t1.c1 = tmp.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +------------PhysicalOlapScan[t2] +------------PhysicalDistribute +--------------PhysicalOlapScan[t3] + +-- !select15 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[LEFT_OUTER_JOIN](t1.c1 = tmp.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +------------PhysicalOlapScan[t2] +------------PhysicalDistribute +--------------PhysicalOlapScan[t3] + +Used: +UnUsed: leading(t1 t2 t3) +SyntaxError: + +-- !select16 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[LEFT_SEMI_JOIN](t1.c1 = t2.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------PhysicalProject +------------PhysicalOlapScan[t2] + +-- !select17 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[RIGHT_SEMI_JOIN](t1.c1 = t2.c2) +--------PhysicalProject +----------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t1] + +Used: leading(t2 t1) +UnUsed: +SyntaxError: + +-- !select18 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------NestedLoopJoin[CROSS_JOIN] +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------PhysicalLimit +------------PhysicalDistribute +--------------PhysicalLimit +----------------PhysicalProject +------------------PhysicalOlapScan[t2] + +-- !select19 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------NestedLoopJoin[CROSS_JOIN] +--------PhysicalProject +----------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t1] + +Used: leading(t2 t1) +UnUsed: +SyntaxError: + +-- !select20 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------NestedLoopJoin[CROSS_JOIN] +--------hashJoin[INNER_JOIN](cte.c1 = cte.c2) +----------PhysicalProject +------------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalProject +--------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t2] + +-- !select21 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------NestedLoopJoin[CROSS_JOIN] +--------NestedLoopJoin[CROSS_JOIN](cte.c1 = cte.c2) +----------PhysicalProject +------------PhysicalOlapScan[t2] +----------PhysicalDistribute +------------PhysicalProject +--------------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------PhysicalOlapScan[t3] + +Used: leading(t2 t1 t3) +UnUsed: +SyntaxError: + +-- !select22 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------PhysicalOlapScan[t2] + +Used: +UnUsed: +SyntaxError: leading(t66 t1) Msg:can not find table: t66 + +-- !select23 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------PhysicalOlapScan[t2] + +Used: +UnUsed: +SyntaxError: leading(t3 t1) Msg:can not find table: t3 + +-- !select24 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------NestedLoopJoin[CROSS_JOIN] +--------hashJoin[INNER_JOIN](cte.c1 = cte.c2) +----------PhysicalProject +------------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalProject +--------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t2] + +Used: +UnUsed: +SyntaxError: leading(t2 cte t1) Msg:Leading alias can only be table name alias + +-- !select25 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +--------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +----------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t3] + +Used: +UnUsed: +SyntaxError: leading(t1 t2) Msg:tables should be same as join tables + +-- !select26 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t2.c2 = t3.c3) +--------hashJoin[INNER_JOIN](t1.c1 = t2.c2) +----------PhysicalOlapScan[t1] +----------PhysicalDistribute +------------PhysicalOlapScan[t2] +--------PhysicalDistribute +----------PhysicalOlapScan[t3] + +Used: +UnUsed: +SyntaxError: leading(t1 t1 t2 t3) Msg:duplicated table + +-- !select27 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t_2.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------PhysicalOlapScan[t2] + +Used: leading(t1 t_2) +UnUsed: +SyntaxError: + +-- !select28 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t_2.c2) +--------PhysicalOlapScan[t1] +--------PhysicalDistribute +----------PhysicalOlapScan[t2] + +Used: +UnUsed: +SyntaxError: leading(t1 t2) Msg:can not find table: t2 + +-- !select29 -- +PhysicalResultSink +--PhysicalDistribute +----PhysicalProject +------hashJoin[INNER_JOIN](t1.c1 = t_1.c1) +--------PhysicalOlapScan[t1] +--------PhysicalOlapScan[t1] + +Used: leading(t1 t_1) +UnUsed: +SyntaxError: diff --git a/regression-test/data/nereids_p0/subquery/test_subquery_in_project.out b/regression-test/data/nereids_p0/subquery/test_subquery_in_project.out new file mode 100644 index 000000000000000..5b97935639059d5 --- /dev/null +++ b/regression-test/data/nereids_p0/subquery/test_subquery_in_project.out @@ -0,0 +1,50 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +3 + +-- !sql2 -- +3 + +-- !sql3 -- +3 + +-- !sql4 -- +false + +-- !sql5 -- +false + +-- !sql6 -- +true + +-- !sql7 -- +2 + +-- !sql8 -- +4 +4 + +-- !sql9 -- +4 +4 + +-- !sql10 -- +false +true + +-- !sql11 -- +false +true + +-- !sql12 -- +true +true + +-- !sql13 -- +2 +2 + +-- !sql14 -- +\N 2.0 +2020-09-09 2.0 + diff --git a/regression-test/data/nereids_p0/update/update_cte.out b/regression-test/data/nereids_p0/update/update_cte.out new file mode 100644 index 000000000000000..24a7d0462fb7208 --- /dev/null +++ b/regression-test/data/nereids_p0/update/update_cte.out @@ -0,0 +1,16 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 10 1 1 1.0 2000-01-01 +2 20 2 2 2.0 2000-01-02 +3 30 5 3 3.0 2000-01-03 + +-- !sql -- +1 10 2 1 2.0 2000-01-01 +2 20 2 2 2.0 2000-01-02 +3 30 5 3 3.0 2000-01-03 + +-- !sql -- +1 10 10 1 1000.0 2000-01-01 +2 20 2 2 2.0 2000-01-02 +3 30 5 3 3.0 2000-01-03 + 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 8c934fb1876e226..0aa36ae310959b0 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 @@ -23,7 +23,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer] --------------PhysicalDistribute -----------------hashJoin[INNER_JOIN](ctr1.ctr_store_sk = ctr2.ctr_store_sk)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) +----------------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)) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN](store.s_store_sk = ctr1.ctr_store_sk) ----------------------PhysicalDistribute @@ -32,11 +32,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------PhysicalProject --------------------------filter((cast(s_state as VARCHAR(*)) = 'SD')) ----------------------------PhysicalOlapScan[store] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 61dc688e5f5cdd3..6e395565dd43edc 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 @@ -42,7 +42,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.00) THEN (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)) ELSE 0.0 END > CASE WHEN (year_total > 0.00) THEN (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)) ELSE 0.0 END) +------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.00) THEN (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)) ELSE 0 END > CASE WHEN (year_total > 0.00) THEN (cast(year_total as DOUBLE) / cast(year_total as DOUBLE)) ELSE 0 END) --------------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) ------------------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 df28c5bee47b661..83982f378271394 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 @@ -24,7 +24,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) +------------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) ----------------PhysicalDistribute ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) @@ -38,11 +38,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------filter((cast(ca_state as VARCHAR(*)) = 'IN')) ----------------------------PhysicalOlapScan[customer_address] --------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 0fe8176d6f271f2..3a2ff5eb3dc3e6f 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 @@ -3,7 +3,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject -------filter((CASE WHEN (mean = 0.0) THEN 0.0 ELSE (stdev / mean) END > 1.0)) +------filter((CASE WHEN (mean = 0) THEN 0 ELSE (stdev / mean) END > 1)) --------hashAgg[GLOBAL] ----------PhysicalDistribute ------------hashAgg[LOCAL] 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 8ba49dc8d60bdd8..b8d6435601a5d9f 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 @@ -14,30 +14,32 @@ PhysicalResultSink ----------------------PhysicalWindow ------------------------PhysicalQuickSort --------------------------PhysicalDistribute -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) +----------------------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------PhysicalWindow ------------------------PhysicalQuickSort --------------------------PhysicalDistribute -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalOlapScan[web_sales] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) +----------------------------------------------PhysicalOlapScan[date_dim] 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 61342aa658f34bc..34a39fb42f31b3d 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 @@ -25,7 +25,7 @@ PhysicalResultSink ------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '501-1000') OR (cast(hd_buy_potential as VARCHAR(*)) = 'Unknown'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1.0)) +--------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '501-1000') OR (cast(hd_buy_potential as VARCHAR(*)) = 'Unknown'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1)) ----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject 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 9cd4ce49056dfd4..370d84a76438e11 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 @@ -42,12 +42,12 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END > CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END) +------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0) THEN (year_total / year_total) ELSE NULL END > CASE WHEN (year_total > 0) THEN (year_total / year_total) ELSE NULL END) --------------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) ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.0)) +----------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0)) ------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject @@ -59,6 +59,6 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0.0)(t_w_firstyear.sale_type = 'w')) +------------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0)(t_w_firstyear.sale_type = 'w')) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 77c7b273ba3d0f4..bfcec6ce4127f4a 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 @@ -24,7 +24,7 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) +------------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) ----------------PhysicalDistribute ------------------PhysicalCteConsumer ( cteId=CTEId#0 ) @@ -38,11 +38,10 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------filter((cast(ca_state as VARCHAR(*)) = 'CA')) ----------------------------PhysicalOlapScan[customer_address] --------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 6114877bc9dd666..9913e27f5fb37b5 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 @@ -9,13 +9,12 @@ PhysicalResultSink ------------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))) -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) -------------------------------PhysicalOlapScan[lineitem] +------------------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 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 index 6114877bc9dd666..9913e27f5fb37b5 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out @@ -9,13 +9,12 @@ PhysicalResultSink ------------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))) -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) -------------------------------PhysicalOlapScan[lineitem] +------------------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 diff --git a/regression-test/data/query_p0/aggregate/bitmap_agg.out b/regression-test/data/query_p0/aggregate/bitmap_agg.out index 82a80101eb94922..f078fe0f6eb597e 100644 --- a/regression-test/data/query_p0/aggregate/bitmap_agg.out +++ b/regression-test/data/query_p0/aggregate/bitmap_agg.out @@ -5,3 +5,13 @@ -- !sql2 -- 20000 +-- !sql3 -- +25 25 25 + +-- !sql4 -- +5 5 +5 5 +5 5 +5 5 +5 5 + diff --git a/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_floor_ceil.out b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_floor_ceil.out new file mode 100644 index 000000000000000..247a44b173d5318 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_floor_ceil.out @@ -0,0 +1,37 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +2023-07-14T10:51:10 + +-- !sql2 -- +2023-07-14T10:50 + +-- !sql3 -- +2023-07-14T09:00 + +-- !sql4 -- +2023-07-10T00:00 + +-- !sql5 -- +2023-07-01T00:00 + +-- !sql6 -- +2021-01-01T00:00 + +-- !sql7 -- +2023-07-14T10:51:15 + +-- !sql8 -- +2023-07-14T10:55 + +-- !sql9 -- +2023-07-14T14:00 + +-- !sql10 -- +2023-07-15T00:00 + +-- !sql11 -- +2023-12-01T00:00 + +-- !sql12 -- +2026-01-01T00:00 + diff --git a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function_regexp.out b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function_regexp.out index 415b8f28221f9db..1684a03a152922c 100644 --- a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function_regexp.out +++ b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function_regexp.out @@ -61,6 +61,12 @@ d -- !sql -- +-- !sql -- + + +-- !sql -- +['ab','c','c','c'] + -- !sql -- a-b-c diff --git a/regression-test/data/query_p0/sql_functions/table_function/explode.out b/regression-test/data/query_p0/sql_functions/table_function/explode.out index c3e327625a2f3c6..2c06dc5d81c99b8 100644 --- a/regression-test/data/query_p0/sql_functions/table_function/explode.out +++ b/regression-test/data/query_p0/sql_functions/table_function/explode.out @@ -46,3 +46,232 @@ 3 1 4 1 +-- !test4 -- +1 0 +1 1 +1 2 +1 3 +1 4 +2 0 +2 1 +2 2 +2 3 +2 4 +3 0 +3 1 +3 2 +3 3 +3 4 +4 0 +4 1 +4 2 +4 3 +4 4 +5 0 +5 1 +5 2 +5 3 +5 4 +6 0 +6 1 +6 2 +6 3 +6 4 +7 0 +7 1 +7 2 +7 3 +7 4 +8 0 +8 1 +8 2 +8 3 +8 4 +9 0 +9 1 +9 2 +9 3 +9 4 +10 0 +10 1 +10 2 +10 3 +10 4 +11 0 +11 1 +11 2 +11 3 +11 4 +12 0 +12 1 +12 2 +12 3 +12 4 +13 0 +13 1 +13 2 +13 3 +13 4 +14 0 +14 1 +14 2 +14 3 +14 4 +15 0 +15 1 +15 2 +15 3 +15 4 + +-- !test5 -- +1 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +2 0 +2 1 +2 2 +2 3 +2 4 +2 5 +2 6 +2 7 +2 8 +2 9 +3 0 +3 1 +3 2 +3 3 +3 4 +3 5 +3 6 +3 7 +3 8 +3 9 +4 0 +4 1 +4 2 +4 3 +4 4 +4 5 +4 6 +4 7 +4 8 +4 9 +5 0 +5 1 +5 2 +5 3 +5 4 +5 5 +5 6 +5 7 +5 8 +5 9 +6 0 +6 1 +6 2 +6 3 +6 4 +6 5 +6 6 +6 7 +6 8 +6 9 +7 0 +7 1 +7 2 +7 3 +7 4 +7 5 +7 6 +7 7 +7 8 +7 9 +8 0 +8 1 +8 2 +8 3 +8 4 +8 5 +8 6 +8 7 +8 8 +8 9 +9 0 +9 1 +9 2 +9 3 +9 4 +9 5 +9 6 +9 7 +9 8 +9 9 +10 0 +10 1 +10 2 +10 3 +10 4 +10 5 +10 6 +10 7 +10 8 +10 9 +11 0 +11 1 +11 2 +11 3 +11 4 +11 5 +11 6 +11 7 +11 8 +11 9 +12 0 +12 1 +12 2 +12 3 +12 4 +12 5 +12 6 +12 7 +12 8 +12 9 +13 0 +13 1 +13 2 +13 3 +13 4 +13 5 +13 6 +13 7 +13 8 +13 9 +14 0 +14 1 +14 2 +14 3 +14 4 +14 5 +14 6 +14 7 +14 8 +14 9 +15 0 +15 1 +15 2 +15 3 +15 4 +15 5 +15 6 +15 7 +15 8 +15 9 + diff --git a/regression-test/data/query_p0/system/test_metadata_name_ids.out b/regression-test/data/query_p0/system/test_metadata_name_ids.out new file mode 100644 index 000000000000000..4dc532f4d21ec2a --- /dev/null +++ b/regression-test/data/query_p0/system/test_metadata_name_ids.out @@ -0,0 +1,14 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !desc -- +CATALOG_ID BIGINT Yes false \N +CATALOG_NAME VARCHAR(512) Yes false \N +DATABASE_ID BIGINT Yes false \N +DATABASE_NAME VARCHAR(64) Yes false \N +TABLE_ID BIGINT Yes false \N +TABLE_NAME VARCHAR(64) Yes false \N + +-- !select2 -- +internal demo test_metadata_name_ids + +-- !select3 -- + diff --git a/regression-test/data/query_p0/system/test_query_sys_tables.out b/regression-test/data/query_p0/system/test_query_sys_tables.out index 73300030c84d656..7b7ce0583de21d4 100644 --- a/regression-test/data/query_p0/system/test_query_sys_tables.out +++ b/regression-test/data/query_p0/system/test_query_sys_tables.out @@ -57,3 +57,4 @@ test_view -- !sql -- +-- !sql -- diff --git a/regression-test/data/view_p0/view_p0.out b/regression-test/data/view_p0/view_p0.out index a90e1f9623c44c0..ff952fdaa7b833f 100644 --- a/regression-test/data/view_p0/view_p0.out +++ b/regression-test/data/view_p0/view_p0.out @@ -12,3 +12,6 @@ 1 2023-08-01 DORID_FIELD1 DORID_FIELD2 ["cat", "dog"] cat 1 2023-08-01 DORID_FIELD1 DORID_FIELD2 ["cat", "dog"] dog +-- !sql -- +960 + 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 8d6af3e946dbd44..2dc4a0c8fab50ef 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 @@ -146,8 +146,8 @@ class StreamLoadAction implements SuiteAction { long startTime = System.currentTimeMillis() try { if (headers.containsKey("version")) { - log.info("Stream load with sql") - def uri = "http://${address.hostString}:${address.port}/api/_stream_load_with_sql" + log.info("http stream") + def uri = "http://${address.hostString}:${address.port}/api/_http_stream" HttpClients.createDefault().withCloseable { client -> RequestBuilder requestBuilder = prepareRequestHeader(RequestBuilder.put(uri)) HttpEntity httpEntity = prepareHttpEntity(client) diff --git a/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyArrayDecimal.java b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyArrayDecimal.java new file mode 100644 index 000000000000000..c689a44af9dcd21 --- /dev/null +++ b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyArrayDecimal.java @@ -0,0 +1,35 @@ +// 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.udf; +import java.math.BigDecimal; + +import org.apache.hadoop.hive.ql.exec.UDF; +import org.apache.log4j.Logger; +import java.util.*; + +public class MyArrayDecimal extends UDF { + private static final Logger LOG = Logger.getLogger(MyArrayDecimal.class); + + public Integer evaluate(ArrayList arr) { + Integer scale = 0; + for (BigDecimal value : arr) { + scale = value.scale(); + } + return scale; + } +} diff --git a/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyMapDecimal.java b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyMapDecimal.java new file mode 100644 index 000000000000000..f33c230b373515f --- /dev/null +++ b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyMapDecimal.java @@ -0,0 +1,35 @@ +// 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.udf; +import java.math.BigDecimal; + +import org.apache.hadoop.hive.ql.exec.UDF; +import org.apache.log4j.Logger; +import java.util.*; + +public class MyMapDecimal extends UDF { + private static final Logger LOG = Logger.getLogger(MyMapDecimal.class); + + public Integer evaluate(HashMap mp) { + Integer scale = 0; + for (Map.Entry value : mp.entrySet()) { + scale = value.getKey().scale() + value.getValue().scale(); + } + return scale; + } +} diff --git a/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyMapRetDecimal.java b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyMapRetDecimal.java new file mode 100644 index 000000000000000..b5c860de940cd00 --- /dev/null +++ b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyMapRetDecimal.java @@ -0,0 +1,42 @@ +// 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.udf; +import java.math.BigDecimal; + +import org.apache.hadoop.hive.ql.exec.UDF; +import org.apache.log4j.Logger; +import java.util.*; + +public class MyMapRetDecimal extends UDF { + private static final Logger LOG = Logger.getLogger(MyMapRetDecimal.class); + + public HashMap evaluate(int id) { + BigDecimal idBigDecimal = new BigDecimal(id); + + BigDecimal result = BigDecimal.ZERO; + result = result.add(idBigDecimal.divide(new BigDecimal("1"))); + result = result.add(idBigDecimal.divide(new BigDecimal("10"))); + result = result.add(idBigDecimal.divide(new BigDecimal("100"))); + result = result.add(idBigDecimal.divide(new BigDecimal("1000"))); + HashMap mp = new HashMap<>(); + for (int i = 0; i < 10; i++) { + mp.put(idBigDecimal, result); + } + return mp; + } +} diff --git a/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyReturnMapString.java b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyReturnMapString.java index a416a8371e4abcb..17daa4e4122672b 100644 --- a/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyReturnMapString.java +++ b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MyReturnMapString.java @@ -36,7 +36,6 @@ public void destroy(State state) { } public void add(State state, Integer k, Double v) { - LOG.info("udaf nest k v " + k + " " + v); state.counter.put(k, v); } diff --git a/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MySumReturnMapIntDou.java b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MySumReturnMapIntDou.java index 7a86666ef3b535e..5e1c8bb2651ff58 100644 --- a/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MySumReturnMapIntDou.java +++ b/regression-test/java-udf-src/src/main/java/org/apache/doris/udf/MySumReturnMapIntDou.java @@ -39,7 +39,6 @@ public void destroy(State state) { } public void add(State state, Integer k, Double v) { - LOG.info("udaf nest k v " + k + " " + v); state.counter.put(k, v); } diff --git a/regression-test/pipeline/p0/conf/regression-conf.groovy b/regression-test/pipeline/p0/conf/regression-conf.groovy index 7464b28d03cabf2..e305e18eff91ef2 100644 --- a/regression-test/pipeline/p0/conf/regression-conf.groovy +++ b/regression-test/pipeline/p0/conf/regression-conf.groovy @@ -54,7 +54,9 @@ testDirectories = "" // this groups will not be executed excludeGroups = "" // this suites will not be executed -excludeSuites = "test_full_compaction,test_default_limit,test_profile,test_broker_load,test_spark_load,test_refresh_mtmv,test_bitmap_filter,test_export_parquet,test_doris_jdbc_catalog,nereids_delete_mow_partial_update,test_hdfs_tvf" + +excludeSuites = "test_leading,test_stream_load_move_memtable,ds_rf95,test_default_limit,test_profile,test_broker_load,test_spark_load,test_refresh_mtmv,test_bitmap_filter,test_export_parquet,test_doris_jdbc_catalog,nereids_delete_mow_partial_update,test_hdfs_tvf" + // this directories will not be executed excludeDirectories = "workload_manager_p1" diff --git a/regression-test/suites/cold_heat_separation/policy/alter.groovy b/regression-test/suites/cold_heat_separation/policy/alter.groovy index 3cb46cde103263a..672f79427630b06 100644 --- a/regression-test/suites/cold_heat_separation/policy/alter.groovy +++ b/regression-test/suites/cold_heat_separation/policy/alter.groovy @@ -185,6 +185,15 @@ suite("alter_policy") { // test when policy binding to resource def has_resource_policy_alter = "has_resource_policy_alter" + sql """ + DROP STORAGE POLICY IF EXISTS has_resouce_policy_alter_policy + """ + sql """ + DROP STORAGE POLICY IF EXISTS has_test_policy_to_alter + """ + sql """ + DROP STORAGE POLICY IF EXISTS has_test_policy_to_alter_1 + """ check_resource_delete_if_exist(has_resource_policy_alter) create_source(has_resource_policy_alter) sql """ diff --git a/regression-test/suites/cold_heat_separation_p2/add_drop_partition.groovy b/regression-test/suites/cold_heat_separation_p2/add_drop_partition.groovy new file mode 100644 index 000000000000000..3bfe7fb8519af42 --- /dev/null +++ b/regression-test/suites/cold_heat_separation_p2/add_drop_partition.groovy @@ -0,0 +1,288 @@ +// 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 groovy.json.JsonSlurper +import org.codehaus.groovy.runtime.IOGroovyMethods +import java.time.LocalDate; + +suite("add_drop_partition") { + def fetchBeHttp = { check_func, meta_url -> + def i = meta_url.indexOf("/api") + String endPoint = meta_url.substring(0, i) + String metaUri = meta_url.substring(i) + i = endPoint.lastIndexOf('/') + endPoint = endPoint.substring(i + 1) + httpTest { + endpoint endPoint + uri metaUri + op "get" + check check_func + } + } + // data_sizes is one arrayList, t is tablet + def fetchDataSize = { data_sizes, t -> + def tabletId = t[0] + String meta_url = t[17] + def clos = { respCode, body -> + logger.info("test ttl expired resp Code {}", "${respCode}".toString()) + assertEquals("${respCode}".toString(), "200") + String out = "${body}".toString() + def obj = new JsonSlurper().parseText(out) + data_sizes[0] = obj.local_data_size + data_sizes[1] = obj.remote_data_size + } + fetchBeHttp(clos, meta_url.replace("header", "data_size")) + } + // used as passing out parameter to fetchDataSize + List sizes = [-1, -1] + def tableName = "tbl1" + sql """ DROP TABLE IF EXISTS ${tableName} """ + + def check_storage_policy_exist = { name-> + def polices = sql""" + show storage policy; + """ + for (p in polices) { + if (name == p[0]) { + return true; + } + } + return false; + } + + def resource_name = "test_add_drop_partition_resource" + def policy_name= "test_add_drop_partition_policy" + + if (check_storage_policy_exist(policy_name)) { + sql """ + DROP STORAGE POLICY ${policy_name} + """ + } + + def has_resouce = sql """ + SHOW RESOURCES WHERE NAME = "${resource_name}"; + """ + if (has_resouce.size() > 0) { + sql """ + DROP RESOURCE ${resource_name} + """ + } + + sql """ + CREATE RESOURCE IF NOT EXISTS "${resource_name}" + PROPERTIES( + "type"="s3", + "AWS_ENDPOINT" = "${getS3Endpoint()}", + "AWS_REGION" = "${getS3Region()}", + "AWS_ROOT_PATH" = "regression/cooldown", + "AWS_ACCESS_KEY" = "${getS3AK()}", + "AWS_SECRET_KEY" = "${getS3SK()}", + "AWS_MAX_CONNECTIONS" = "50", + "AWS_REQUEST_TIMEOUT_MS" = "3000", + "AWS_CONNECTION_TIMEOUT_MS" = "1000", + "AWS_BUCKET" = "${getS3BucketName()}", + "s3_validity_check" = "true" + ); + """ + + sql """ + CREATE STORAGE POLICY IF NOT EXISTS ${policy_name} + PROPERTIES( + "storage_resource" = "${resource_name}", + "cooldown_ttl" = "300" + ) + """ + + // test one replica + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE ${tableName} ( + `k1` int, + `k2` date + ) + PARTITION BY RANGE(k2)( + partition p1 VALUES LESS THAN ("2014-01-01"), + partition p2 VALUES LESS THAN ("2015-01-01"), + partition p3 VALUES LESS THAN ("2016-01-01") + ) + DISTRIBUTED BY HASH(k1) BUCKETS 1 + PROPERTIES + ( + "replication_num" = "1", + "storage_policy" = "${policy_name}" + ); + """ + sql """ + insert into ${tableName} values(1, "2013-01-01"); + """ + sql """ + insert into ${tableName} values(1, "2014-01-01"); + """ + sql """ + insert into ${tableName} values(1, "2015-01-01"); + """ + + // show tablets from table, 获取第一个tablet的 LocalDataSize1 + def tablets = sql """ + SHOW TABLETS FROM ${tableName} + """ + log.info( "test tablets not empty") + assertTrue(tablets.size() > 0) + fetchDataSize(sizes, tablets[0]) + def LocalDataSize1 = sizes[0] + def RemoteDataSize1 = sizes[1] + log.info( "test local size {} not zero, remote size {}", LocalDataSize1, RemoteDataSize1) + assertTrue(LocalDataSize1 != 0) + log.info( "test remote size is zero") + assertEquals(RemoteDataSize1, 0) + def originLocalDataSize1 = LocalDataSize1; + + // 等待10min,show tablets from table, 预期not_use_storage_policy_tablet_list 的 RemoteDataSize 为0,LocalDataSize不为0 + sleep(600000) + + + tablets = sql """ + SHOW TABLETS FROM ${tableName} + """ + log.info( "test tablets not empty") + fetchDataSize(sizes, tablets[0]) + while (sizes[1] == 0) { + log.info( "test remote size is zero, sleep 10s") + sleep(10000) + tablets = sql """ + SHOW TABLETS FROM ${tableName} + """ + fetchDataSize(sizes, tablets[0]) + } + assertTrue(tablets.size() > 0) + LocalDataSize1 = sizes[0] + RemoteDataSize1 = sizes[1] + Long sleepTimes = 0; + while (RemoteDataSize1 != originLocalDataSize1 && sleepTimes < 60) { + log.info( "test remote size is same with origin size, sleep 10s") + sleep(10000) + tablets = sql """ + SHOW TABLETS FROM + """ + fetchDataSize(sizes, tablets[0]) + LocalDataSize1 = sizes[0] + RemoteDataSize1 = sizes[1] + sleepTimes += 1 + } + log.info( "test local size is zero") + assertEquals(LocalDataSize1, 0) + log.info( "test remote size not zero") + assertEquals(RemoteDataSize1, originLocalDataSize1) + + // 12列是storage policy + def partitions = sql "show partitions from ${tableName}" + for (par in partitions) { + assertTrue(par[12] == "${policy_name}") + } + + try_sql """ + drop storage policy add_policy; + """ + + try_sql """ + drop resource add_resource; + """ + + sql """ + CREATE RESOURCE IF NOT EXISTS "add_resource" + PROPERTIES( + "type"="s3", + "AWS_ENDPOINT" = "${getS3Endpoint()}", + "AWS_REGION" = "${getS3Region()}", + "AWS_ROOT_PATH" = "regression/cooldown", + "AWS_ACCESS_KEY" = "${getS3AK()}", + "AWS_SECRET_KEY" = "${getS3SK()}", + "AWS_MAX_CONNECTIONS" = "50", + "AWS_REQUEST_TIMEOUT_MS" = "3000", + "AWS_CONNECTION_TIMEOUT_MS" = "1000", + "AWS_BUCKET" = "${getS3BucketName()}", + "s3_validity_check" = "true" + ); + """ + + try_sql """ + create storage policy tmp_policy + PROPERTIES( "storage_resource" = "add_resource", "cooldown_ttl" = "300"); + """ + + // can not set to one policy with different resource + try { + sql """alter table ${tableName} set ("storage_policy" = "add_policy");""" + } catch (java.sql.SQLException t) { + assertTrue(true) + } + + sql """ + CREATE STORAGE POLICY IF NOT EXISTS add_policy1 + PROPERTIES( + "storage_resource" = "${resource_name}", + "cooldown_ttl" = "60" + ) + """ + + sql """alter table ${tableName} set ("storage_policy" = "add_policy1");""" + + // wait for report + sleep(300000) + + partitions = sql "show partitions from ${tableName}" + for (par in partitions) { + assertTrue(par[12] == "add_policy1") + } + + + sql """ + alter table ${tableName} ADD PARTITION np + VALUES LESS THAN ("2016-01-01"); + """ + + sql """ + insert into ${tableName} values(1, "2016-01-01"); + """ + + partitions = sql "show partitions from ${tableName}" + for (par in partitions) { + assertTrue(par[12] == "add_policy1") + } + + sql """ + sql * from ${tableName} + """ + + sql """ + DROP TABLE ${tableName} + """ + + sql """ + drop storage policy add_policy; + """ + + sql """ + drop storage policy add_policy1; + """ + + sql """ + drop resource add_resource; + """ + + + +} \ No newline at end of file diff --git a/regression-test/suites/cold_heat_separation_p2/create_table_use_dynamic_partition.groovy b/regression-test/suites/cold_heat_separation_p2/create_table_use_dynamic_partition.groovy new file mode 100644 index 000000000000000..e5ae29293cbfa9f --- /dev/null +++ b/regression-test/suites/cold_heat_separation_p2/create_table_use_dynamic_partition.groovy @@ -0,0 +1,283 @@ +// 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 groovy.json.JsonSlurper +import org.codehaus.groovy.runtime.IOGroovyMethods +import java.time.LocalDate; + +suite("cold_heat_dynamic_partition") { + def fetchBeHttp = { check_func, meta_url -> + def i = meta_url.indexOf("/api") + String endPoint = meta_url.substring(0, i) + String metaUri = meta_url.substring(i) + i = endPoint.lastIndexOf('/') + endPoint = endPoint.substring(i + 1) + httpTest { + endpoint endPoint + uri metaUri + op "get" + check check_func + } + } + // data_sizes is one arrayList, t is tablet + def fetchDataSize = { data_sizes, t -> + def tabletId = t[0] + String meta_url = t[17] + def clos = { respCode, body -> + logger.info("test ttl expired resp Code {}", "${respCode}".toString()) + assertEquals("${respCode}".toString(), "200") + String out = "${body}".toString() + def obj = new JsonSlurper().parseText(out) + data_sizes[0] = obj.local_data_size + data_sizes[1] = obj.remote_data_size + } + fetchBeHttp(clos, meta_url.replace("header", "data_size")) + } + // used as passing out parameter to fetchDataSize + List sizes = [-1, -1] + def tableName = "tbl2" + sql """ DROP TABLE IF EXISTS ${tableName} """ + + def check_storage_policy_exist = { name-> + def polices = sql""" + show storage policy; + """ + for (p in polices) { + if (name == p[0]) { + return true; + } + } + return false; + } + + def resource_name = "test_dynamic_partition_resource" + def policy_name= "test_dynamic_partition_policy" + + if (check_storage_policy_exist(policy_name)) { + sql """ + DROP STORAGE POLICY ${policy_name} + """ + } + + def has_resouce = sql """ + SHOW RESOURCES WHERE NAME = "${resource_name}"; + """ + if (has_resouce.size() > 0) { + sql """ + DROP RESOURCE IF EXISTS ${resource_name} + """ + } + + sql """ + CREATE RESOURCE IF NOT EXISTS "${resource_name}" + PROPERTIES( + "type"="s3", + "AWS_ENDPOINT" = "${getS3Endpoint()}", + "AWS_REGION" = "${getS3Region()}", + "AWS_ROOT_PATH" = "regression/cooldown", + "AWS_ACCESS_KEY" = "${getS3AK()}", + "AWS_SECRET_KEY" = "${getS3SK()}", + "AWS_MAX_CONNECTIONS" = "50", + "AWS_REQUEST_TIMEOUT_MS" = "3000", + "AWS_CONNECTION_TIMEOUT_MS" = "1000", + "AWS_BUCKET" = "${getS3BucketName()}", + "s3_validity_check" = "true" + ); + """ + + sql """ + CREATE STORAGE POLICY IF NOT EXISTS ${policy_name} + PROPERTIES( + "storage_resource" = "${resource_name}", + "cooldown_ttl" = "300" + ) + """ + + // test one replica + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE ${tableName} ( + `k1` int, + `k2` date + ) + PARTITION BY RANGE(k2)() + DISTRIBUTED BY HASH(k1) BUCKETS 1 + PROPERTIES + ( + "dynamic_partition.enable" = "true", + "dynamic_partition.time_unit" = "DAY", + "dynamic_partition.end" = "3", + "dynamic_partition.prefix" = "p", + "dynamic_partition.buckets" = "1", + "dynamic_partition.replication_num" = "1", + "dynamic_partition.start" = "-3", + "storage_policy" = "${policy_name}", + "replication_num" = "1" + ); + """ + LocalDate currentDate = LocalDate.now(); + LocalDate currentDatePlusOne = currentDate.plusDays(1); + LocalDate currentDatePlusTwo = currentDate.plusDays(2); + LocalDate currentDatePlusThree = currentDate.plusDays(3); + sql """ + insert into ${tableName} values(1, "${currentDate.toString()}"); + """ + sql """ + insert into ${tableName} values(1, "${currentDatePlusOne.toString()}"); + """ + sql """ + insert into ${tableName} values(1, "${currentDatePlusTwo.toString()}"); + """ + sql """ + insert into ${tableName} values(1, "${currentDatePlusThree.toString()}"); + """ + + // show tablets from table, 获取第一个tablet的 LocalDataSize1 + def tablets = sql """ + SHOW TABLETS FROM ${tableName} + """ + log.info( "test tablets not empty") + assertTrue(tablets.size() > 0) + fetchDataSize(sizes, tablets[0]) + def LocalDataSize1 = sizes[0] + def RemoteDataSize1 = sizes[1] + log.info( "test local size {} not zero, remote size {}", LocalDataSize1, RemoteDataSize1) + assertTrue(LocalDataSize1 != 0) + log.info( "test remote size is zero") + assertEquals(RemoteDataSize1, 0) + def originLocalDataSize1 = LocalDataSize1; + + // 等待10min,show tablets from table, 预期not_use_storage_policy_tablet_list 的 RemoteDataSize 为0,LocalDataSize不为0 + sleep(600000) + + + tablets = sql """ + SHOW TABLETS FROM ${tableName} + """ + log.info( "test tablets not empty") + fetchDataSize(sizes, tablets[0]) + while (sizes[1] == 0) { + log.info( "test remote size is zero, sleep 10s") + sleep(10000) + tablets = sql """ + SHOW TABLETS FROM ${tableName} + """ + fetchDataSize(sizes, tablets[0]) + } + assertTrue(tablets.size() > 0) + LocalDataSize1 = sizes[0] + RemoteDataSize1 = sizes[1] + Long sleepTimes = 0; + while (RemoteDataSize1 != originLocalDataSize1 && sleepTimes < 60) { + log.info( "test remote size is same with origin size, sleep 10s") + sleep(10000) + tablets = sql """ + SHOW TABLETS FROM + """ + fetchDataSize(sizes, tablets[0]) + LocalDataSize1 = sizes[0] + RemoteDataSize1 = sizes[1] + sleepTimes += 1 + } + log.info( "test local size is zero") + assertEquals(LocalDataSize1, 0) + log.info( "test remote size not zero") + assertEquals(RemoteDataSize1, originLocalDataSize1) + + // 12列是storage policy + def partitions = sql "show partitions from ${tableName}" + for (par in partitions) { + assertTrue(par[12] == "${policy_name}") + } + + try_sql """ + drop storage policy tmp_policy; + """ + + try_sql """ + drop resource tmp_resource; + """ + + sql """ + CREATE RESOURCE IF NOT EXISTS "tmp_resource" + PROPERTIES( + "type"="s3", + "AWS_ENDPOINT" = "${getS3Endpoint()}", + "AWS_REGION" = "${getS3Region()}", + "AWS_ROOT_PATH" = "regression/cooldown", + "AWS_ACCESS_KEY" = "${getS3AK()}", + "AWS_SECRET_KEY" = "${getS3SK()}", + "AWS_MAX_CONNECTIONS" = "50", + "AWS_REQUEST_TIMEOUT_MS" = "3000", + "AWS_CONNECTION_TIMEOUT_MS" = "1000", + "AWS_BUCKET" = "${getS3BucketName()}", + "s3_validity_check" = "true" + ); + """ + + try_sql """ + create storage policy tmp_policy + PROPERTIES( "storage_resource" = "tmp_resource", "cooldown_ttl" = "300"); + """ + + // can not set to one policy with different resource + try { + sql """alter table ${tableName} set ("storage_policy" = "tmp_policy");""" + } catch (java.sql.SQLException t) { + assertTrue(true) + } + + sql """ + CREATE STORAGE POLICY IF NOT EXISTS tmp_policy1 + PROPERTIES( + "storage_resource" = "${resource_name}", + "cooldown_ttl" = "60" + ) + """ + + sql """alter table ${tableName} set ("storage_policy" = "tmp_policy1");""" + + // wait for report + sleep(300000) + + partitions = sql "show partitions from ${tableName}" + for (par in partitions) { + assertTrue(par[12] == "tmp_policy1") + } + + sql """ + sql * from ${tableName} + """ + + sql """ + DROP TABLE ${tableName} + """ + + sql """ + drop storage policy tmp_policy; + """ + + sql """ + drop storage policy tmp_policy1; + """ + + sql """ + drop resource tmp_resource; + """ + + + +} \ No newline at end of file diff --git a/regression-test/suites/compaction/test_compaction_cumu_delete.groovy b/regression-test/suites/compaction/test_compaction_cumu_delete.groovy index 5d24f4adfe4b2e5..31f1331969e16e2 100644 --- a/regression-test/suites/compaction/test_compaction_cumu_delete.groovy +++ b/regression-test/suites/compaction/test_compaction_cumu_delete.groovy @@ -50,13 +50,23 @@ suite("test_compaction_cumu_delete") { assert configList instanceof List boolean disableAutoCompaction = true + boolean allowDeleteWhenCumu = false for (Object ele in (List) configList) { assert ele instanceof List if (((List) ele)[0] == "disable_auto_compaction") { disableAutoCompaction = Boolean.parseBoolean(((List) ele)[2]) } + if (((List) ele)[0] == "enable_delete_when_cumu_compaction") { + allowDeleteWhenCumu = Boolean.parseBoolean(((List) ele)[2]) + } + } + + if (!allowDeleteWhenCumu) { + logger.info("Skip test compaction when cumu compaction because not enabled this config") + return } + def triggerCompaction = { be_host, be_http_port, compact_type -> // trigger compactions for all tablets in ${tableName} String tablet_id = tablet[0] diff --git a/regression-test/suites/compaction/test_full_compaction.groovy b/regression-test/suites/compaction/test_full_compaction.groovy index 41813fde3b56b74..eaebd33377d7d54 100644 --- a/regression-test/suites/compaction/test_full_compaction.groovy +++ b/regression-test/suites/compaction/test_full_compaction.groovy @@ -95,7 +95,7 @@ suite("test_full_compaction") { sql "SET skip_delete_bitmap = true" // show all hidden data // (1,10)(1,100)(2,2)(2,20)(2,200)(3,300)(3,100) - qt_skip_delete """select * from ${tableName} order by user_id""" + qt_skip_delete """select * from ${tableName} order by user_id, value""" //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,PathHash,MetaUrl,CompactionStatus String[][] tablets = sql """ show tablets from ${tableName}; """ diff --git a/regression-test/suites/correctness_p0/test_inlineview_error_msg.groovy b/regression-test/suites/correctness_p0/test_inlineview_error_msg.groovy new file mode 100644 index 000000000000000..700498b16ff0364 --- /dev/null +++ b/regression-test/suites/correctness_p0/test_inlineview_error_msg.groovy @@ -0,0 +1,80 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_inlineview_error_msg") { + sql "set enable_nereids_planner=false" + sql """ + drop table if exists tmp_can_drop_t1; + """ + + sql """ + drop table if exists tmp_can_drop_t2; + """ + + sql """ + create table tmp_can_drop_t1 ( + cust_id varchar(96), + user_id varchar(96) + ) + DISTRIBUTED by random BUCKETS 1 + PROPERTIES( + "replication_num" = "1" + ); + """ + + sql """ + create table tmp_can_drop_t2 ( + cust_id varchar(96), + usr_id varchar(96) + ) + DISTRIBUTED by random BUCKETS 1 + PROPERTIES( + "replication_num" = "1" + ); + """ + test { + sql """ + select + a.cust_id, + a.usr_id + from ( + select + a.cust_id, + a.usr_id, + a.user_id + from tmp_can_drop_t1 a + full join ( + select + cust_id, + usr_id + from + tmp_can_drop_t2 + ) b + on b.cust_id = a.cust_id + ) a; + """ + exception "Unknown column 'usr_id' in 'a'" + } + + sql """ + drop table if exists tmp_can_drop_t1; + """ + + sql """ + drop table if exists tmp_can_drop_t2; + """ +} diff --git a/regression-test/suites/correctness_p0/test_push_conjuncts_inlineview.groovy b/regression-test/suites/correctness_p0/test_push_conjuncts_inlineview.groovy index 4eff13264942eee..1276b58807dedaa 100644 --- a/regression-test/suites/correctness_p0/test_push_conjuncts_inlineview.groovy +++ b/regression-test/suites/correctness_p0/test_push_conjuncts_inlineview.groovy @@ -128,6 +128,28 @@ sql """ WHERE dd.d1 IN ('-1'); """ +explain { + sql("""SELECT max(b_key) + FROM + (SELECT a_key, + max(b_key) AS b_key + FROM + (SELECT a_key, + max(b_key) AS b_key + FROM push_conjunct_table + GROUP BY a_key + UNION all + SELECT a_key, + max(b_key) AS b_key + FROM push_conjunct_table + GROUP BY a_key) t2 + GROUP BY t2.a_key ) t + WHERE t.a_key = "abcd" + GROUP BY t.a_key;""") + notContains "having" + contains "= 'abcd'" + } + sql """ DROP TABLE IF EXISTS `push_conjunct_table` """ } diff --git a/regression-test/suites/datatype_p0/decimalv3/test_decimalv3_overflow.groovy b/regression-test/suites/datatype_p0/decimalv3/test_decimalv3_overflow.groovy new file mode 100644 index 000000000000000..bb4b4ba42d736ed --- /dev/null +++ b/regression-test/suites/datatype_p0/decimalv3/test_decimalv3_overflow.groovy @@ -0,0 +1,50 @@ +// 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_decimalv3_overflow") { + def tblName1 = "test_decimalv3_overflow1" + sql "drop table if exists ${tblName1}" + sql """ CREATE TABLE ${tblName1} ( + `data_time` date NOT NULL COMMENT "", + `c1` decimal(22, 4) NULL COMMENT "" + ) ENGINE=OLAP + UNIQUE KEY(`data_time`) + DISTRIBUTED BY HASH(`data_time`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); """ + sql "insert into ${tblName1} values('2022-08-01', 104665062791137173.7169)" + + def tblName2 = "test_decimalv3_overflow2" + sql "drop table if exists ${tblName2}" + sql """ CREATE TABLE ${tblName2} ( + `data_time` date NOT NULL COMMENT "", + `c2` decimal(20, 2) NULL COMMENT "", + ) ENGINE=OLAP + UNIQUE KEY(`data_time`) + DISTRIBUTED BY HASH(`data_time`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); """ + sql "insert into ${tblName2} values('2022-08-01', 705091149953414452.46)" + + qt_sql """ select c2 / 10000 * c1 from ${tblName1}, ${tblName2}; """ + + sql """ set check_overflow_for_decimal=true; """ + + qt_sql """ select c2 / 10000 * c1 from ${tblName1}, ${tblName2}; """ +} diff --git a/regression-test/suites/export_p0/test_export_basic.groovy b/regression-test/suites/export_p0/test_export_basic.groovy index 162b63065e37114..84ff0c30a97fa22 100644 --- a/regression-test/suites/export_p0/test_export_basic.groovy +++ b/regression-test/suites/export_p0/test_export_basic.groovy @@ -332,8 +332,9 @@ suite("test_export_basic", "p0") { check_path_exists.call("${outFilePath}") // exec export + // TODO(ftw): EXPORT TABLE ${table_export_name} PARTITION (more_than_70) where id >100 sql """ - EXPORT TABLE ${table_export_name} PARTITION (more_than_70) where id >100 + EXPORT TABLE ${table_export_name} PARTITION (more_than_70) TO "file://${outFilePath}/" PROPERTIES( "label" = "${label}", @@ -376,7 +377,7 @@ suite("test_export_basic", "p0") { log.info("Stream load result: ${result}".toString()) def json = parseJson(result) assertEquals("success", json.Status.toLowerCase()) - assertEquals(50, json.NumberTotalRows) + assertEquals(81, json.NumberTotalRows) assertEquals(0, json.NumberFilteredRows) } } diff --git a/regression-test/suites/export_p0/test_export_csv.groovy b/regression-test/suites/export_p0/test_export_csv.groovy index a8f75f21a4402b7..ed0f8905bc70832 100644 --- a/regression-test/suites/export_p0/test_export_csv.groovy +++ b/regression-test/suites/export_p0/test_export_csv.groovy @@ -217,8 +217,9 @@ suite("test_export_csv", "p0") { check_path_exists.call("${outFilePath}") // exec export + // TODO(ftw): EXPORT TABLE ${table_export_name} where user_id <11 TO "file://${outFilePath}/" sql """ - EXPORT TABLE ${table_export_name} where user_id <11 TO "file://${outFilePath}/" + EXPORT TABLE ${table_export_name} TO "file://${outFilePath}/" PROPERTIES( "label" = "${label}", "format" = "csv", @@ -273,7 +274,7 @@ suite("test_export_csv", "p0") { log.info("Stream load result: ${result}".toString()) def json = parseJson(result) assertEquals("success", json.Status.toLowerCase()) - assertEquals(10, json.NumberTotalRows) + assertEquals(100, json.NumberTotalRows) assertEquals(0, json.NumberFilteredRows) } } @@ -294,8 +295,9 @@ suite("test_export_csv", "p0") { check_path_exists.call("${outFilePath}") // exec export + // TODO(ftw): EXPORT TABLE ${table_export_name} where user_id <11 TO "file://${outFilePath}/" sql """ - EXPORT TABLE ${table_export_name} where user_id <11 TO "file://${outFilePath}/" + EXPORT TABLE ${table_export_name} TO "file://${outFilePath}/" PROPERTIES( "label" = "${label}", "format" = "csv_with_names", @@ -351,7 +353,7 @@ suite("test_export_csv", "p0") { log.info("Stream load result: ${result}".toString()) def json = parseJson(result) assertEquals("success", json.Status.toLowerCase()) - assertEquals(10, json.NumberTotalRows) + assertEquals(100, json.NumberTotalRows) assertEquals(0, json.NumberFilteredRows) } } @@ -359,8 +361,8 @@ suite("test_export_csv", "p0") { qt_select_load3 """ SELECT * FROM ${table_load_name} t ORDER BY user_id; """ } finally { - // try_sql("DROP TABLE IF EXISTS ${table_load_name}") - // delete_files.call("${outFilePath}") + try_sql("DROP TABLE IF EXISTS ${table_load_name}") + delete_files.call("${outFilePath}") } // 4. test csv_with_names_and_types @@ -372,8 +374,9 @@ suite("test_export_csv", "p0") { check_path_exists.call("${outFilePath}") // exec export + // TODO(ftw): EXPORT TABLE ${table_export_name} where user_id <11 TO "file://${outFilePath}/" sql """ - EXPORT TABLE ${table_export_name} where user_id <11 TO "file://${outFilePath}/" + EXPORT TABLE ${table_export_name} TO "file://${outFilePath}/" PROPERTIES( "label" = "${label}", "format" = "csv_with_names_and_types", @@ -429,7 +432,7 @@ suite("test_export_csv", "p0") { log.info("Stream load result: ${result}".toString()) def json = parseJson(result) assertEquals("success", json.Status.toLowerCase()) - assertEquals(10, json.NumberTotalRows) + assertEquals(100, json.NumberTotalRows) assertEquals(0, json.NumberFilteredRows) } } @@ -437,8 +440,8 @@ suite("test_export_csv", "p0") { qt_select_load4 """ SELECT * FROM ${table_load_name} t ORDER BY user_id; """ } finally { - // try_sql("DROP TABLE IF EXISTS ${table_load_name}") - // delete_files.call("${outFilePath}") + try_sql("DROP TABLE IF EXISTS ${table_load_name}") + delete_files.call("${outFilePath}") } try_sql("DROP TABLE IF EXISTS ${table_export_name}") diff --git a/regression-test/suites/export_p2/test_export_big_data.groovy b/regression-test/suites/export_p2/test_export_big_data.groovy index 0547386851f60e7..a6841c9ab6a0e7b 100644 --- a/regression-test/suites/export_p2/test_export_big_data.groovy +++ b/regression-test/suites/export_p2/test_export_big_data.groovy @@ -56,27 +56,51 @@ suite("test_export_big_data", "p2") { String region = getS3Region() String bucket = context.config.otherConfigs.get("s3BucketName"); + + def delete_files = { dir_path -> + File path = new File(dir_path) + if (path.exists()) { + for (File f: path.listFiles()) { + f.delete(); + } + path.delete(); + } + } + + def table_export_name = "test_export_big_data" // create table and insert sql """ DROP TABLE IF EXISTS ${table_export_name} """ sql """ - CREATE TABLE IF NOT EXISTS ${table_export_name} ( - `id` int(11) NULL, - `name` string NULL, - `age` largeint(11) NULL, - `dt` date NULL, - `dt2` datev2 NULL, - `dtime` datetime NULL, - `dtime2` datetimev2 NULL - ) - DISTRIBUTED BY HASH(id) PROPERTIES("replication_num" = "1"); + CREATE TABLE ${table_export_name} ( + `user_id` largeint(40) NOT NULL COMMENT 'id', + `date` date NOT NULL, + `datetime` datetime NOT NULL, + `city` varchar(20) NULL, + `age` int(11) NULL, + `sex` int(11) NULL, + `bool_col` boolean NULL, + `int_col` int(11) NULL, + `bigint_col` bigint(20) NULL, + `largeint_col` largeint(40) NULL, + `float_col` float NULL, + `double_col` double NULL, + `char_col` char(10) NULL, + `decimal_col` DECIMAL NULL + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `date`, `datetime`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`user_id`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); """ sql """ INSERT INTO ${table_export_name} select * from s3( - "uri" = "https://${bucket}.${s3_endpoint}/regression/export_p2/export_orc/test_export_big_data_dataset.csv", + "uri" = "https://${bucket}.${s3_endpoint}/regression/export_p2/export_orc/test_export_big_data_dataset.orc", "s3.access_key"= "${ak}", "s3.secret_key" = "${sk}", - "format" = "csv"); + "format" = "orc"); """ def uuid = UUID.randomUUID().toString() @@ -87,24 +111,28 @@ suite("test_export_big_data", "p2") { EXPORT TABLE ${table_export_name} TO "file://${outFilePath}/" PROPERTIES( "label" = "${uuid}", - "format" = "orc", - "column_separator"="," + "format" = "orc" ); """ - while (true) { - def res = sql """ show export where label = "${uuid}" """ - logger.info("export state: " + res[0][2]) - if (res[0][2] == "FINISHED") { - def json = parseJson(res[0][11]) - assert json instanceof List - assertEquals("1", json.fileNumber[0]) - log.info("outfile_path: ${json.url[0]}") - return json.url[0]; - } else if (res[0][2] == "CANCELLED") { - throw new IllegalStateException("""export failed: ${res[0][10]}""") - } else { - sleep(5000) + try { + while (true) { + def res = sql """ show export where label = "${uuid}" """ + logger.info("export state: " + res[0][2]) + if (res[0][2] == "FINISHED") { + def json = parseJson(res[0][11]) + assert json instanceof List + assertEquals("1", json.fileNumber[0][0]) + log.info("outfile_path: ${json.url[0][0]}") + return json.url[0][0]; + } else if (res[0][2] == "CANCELLED") { + throw new IllegalStateException("""export failed: ${res[0][10]}""") + } else { + sleep(5000) + } } + } finally { + try_sql("DROP TABLE IF EXISTS ${table_export_name}") + delete_files.call("${outFilePath}") } } diff --git a/regression-test/suites/export_p2/test_outfile_orc_max_file_size.groovy b/regression-test/suites/export_p2/test_outfile_orc_max_file_size.groovy index 71c5980d1b3eb59..3f9abe2c2b29061 100644 --- a/regression-test/suites/export_p2/test_outfile_orc_max_file_size.groovy +++ b/regression-test/suites/export_p2/test_outfile_orc_max_file_size.groovy @@ -49,7 +49,7 @@ suite("test_outfile_orc_max_file_size", "p2") { """ } - def table_export_name = "test_export_max_file_size" + def table_export_name = "test_outfile_orc_max_file_size" create_table(table_export_name) diff --git a/regression-test/suites/external_table_p0/hive/test_partial_update_broker_load.groovy b/regression-test/suites/external_table_p0/hive/test_partial_update_broker_load.groovy new file mode 100644 index 000000000000000..cf7243e5ddaf362 --- /dev/null +++ b/regression-test/suites/external_table_p0/hive/test_partial_update_broker_load.groovy @@ -0,0 +1,92 @@ + +// 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_primary_key_partial_update_broker_load", "p0,external,hive,external_docker,external_docker_hive") { + + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + brokerName = getBrokerName() + hdfsUser = getHdfsUser() + hdfsPasswd = getHdfsPasswd() + hdfs_port = context.config.otherConfigs.get("hdfs_port") + externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + def load_from_hdfs = {testTable, label, hdfsFilePath, format, brokerName, hdfsUser, hdfsPasswd -> + def result1= sql """ + LOAD LABEL ${label} ( + DATA INFILE("${hdfsFilePath}") + INTO TABLE ${testTable} + COLUMNS TERMINATED BY "," + (id, score) + ) + with HDFS ( + "fs.defaultFS"="hdfs://${externalEnvIp}:${hdfs_port}", + "hadoop.username"="${hdfsUser}") + PROPERTIES ( + "timeout"="1200", + "max_filter_ratio"="0", + "partial_columns"="true"); + """ + assertTrue(result1.size() == 1) + assertTrue(result1[0].size() == 1) + assertTrue(result1[0][0] == 0, "Query OK, 0 rows affected") + } + + def wait_for_load_result = {checklabel, testTable -> + max_try_milli_secs = 10000 + while(max_try_milli_secs) { + result = sql "show load where label = '${checklabel}'" + if(result[0][2] == "FINISHED") { + break + } else { + sleep(1000) // wait 1 second every time + max_try_milli_secs -= 1000 + if(max_try_milli_secs <= 0) { + assertEquals(1, 2) + } + } + } + } + + def tableName = "test_primary_key_partial_update_broker_load" + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE ${tableName} ( + `id` int(11) NOT NULL COMMENT "用户 ID", + `name` varchar(65533) NULL COMMENT "用户姓名", + `score` int(11) NOT NULL COMMENT "用户得分", + `test` int(11) NULL COMMENT "null test", + `dft` int(11) DEFAULT "4321") + UNIQUE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true") + """ + sql """insert into ${tableName} values(2, "bob", 2000, 223, 2),(1, "alice", 1000, 123, 1),(3, "tom", 3000, 323, 3);""" + qt_sql """ select * from ${tableName} order by id; """ + def test_load_label = UUID.randomUUID().toString().replaceAll("-", "") + load_from_hdfs(tableName, test_load_label, "hdfs://${externalEnvIp}:${hdfs_port}/user/doris/preinstalled_data/data_case/partial_update/update.csv", "csv", brokerName, hdfsUser, hdfsPasswd) + wait_for_load_result(test_load_label, tableName) + qt_sql """select * from ${tableName} order by id;""" + + sql "set enable_unified_load=true;" + sql "sync;" + def test_load_label2 = UUID.randomUUID().toString().replaceAll("-", "") + load_from_hdfs(tableName, test_load_label2, "hdfs://${externalEnvIp}:${hdfs_port}/user/doris/preinstalled_data/data_case/partial_update/update2.csv", "csv", brokerName, hdfsUser, hdfsPasswd) + wait_for_load_result(test_load_label2, tableName) + qt_sql """select * from ${tableName} order by id;""" + sql "drop table if exists ${tableName};" + } +} diff --git a/regression-test/suites/external_table_p0/jdbc/test_sqlserver_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_sqlserver_jdbc_catalog.groovy index 00bf5cde103e68b..6ced9b7d53a063e 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_sqlserver_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_sqlserver_jdbc_catalog.groovy @@ -70,6 +70,7 @@ suite("test_sqlserver_jdbc_catalog", "p0,external,sqlserver,external_docker,exte order_qt_filter1 """ select * from test_char where 1 = 1 order by id; """ order_qt_filter2 """ select * from test_char where 1 = 1 and id = 1 order by id; """ order_qt_filter3 """ select * from test_char where id = 1 order by id; """ + order_qt_id """ select count(*) from (select * from t_id) as a; """ sql """ drop catalog if exists ${catalog_name} """ diff --git a/regression-test/suites/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.groovy b/regression-test/suites/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.groovy index fbe8fb5597f5033..04feab78ddb80ee 100644 --- a/regression-test/suites/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.groovy +++ b/regression-test/suites/external_table_p0/tvf/test_local_tvf_with_complex_type_insertinto_doris.groovy @@ -19,25 +19,34 @@ import org.junit.Assert // This suit test the `backends` tvf suite("test_local_tvf_with_complex_type_insertinto_doris", "p0") { + sql """set enable_nereids_planner=false""" List> backends = sql """ select * from backends(); """ assertTrue(backends.size() > 0) def be_id = backends[0][0] def dataFilePath = context.config.dataPath + "/external_table_p0/tvf/" def table_name = "comp" + def outFilePath= dataFilePath + if (backends.size() > 1) { // cluster mode need to make sure all be has this data - def outFilePath="/" - def transFile01="${dataFilePath}/comp.orc" - def transFile02="${dataFilePath}/comp.parquet" + outFilePath="/" + def transFile01 = "${dataFilePath}/comp.orc" + def transFile02 = "${dataFilePath}/comp.parquet" + def transFile1="${dataFilePath}/comp_arr.orc" + def transFile2="${dataFilePath}/comp_arr.parquet" for (List backend : backends) { def be_host = backend[1] - scpFiles ("root", be_host, transFile01, outFilePath, false); - scpFiles ("root", be_host, transFile02, outFilePath, false); + scpFiles("root", be_host, transFile01, outFilePath, false) + scpFiles("root", be_host, transFile02, outFilePath, false) + scpFiles ("root", be_host, transFile1, outFilePath, false) + scpFiles ("root", be_host, transFile2, outFilePath, false); } + } qt_sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" // create doris table + sql """ DROP TABLE IF EXISTS ${table_name} """ qt_sql """ CREATE TABLE IF NOT EXISTS ${table_name} ( `id` int(11) NULL, @@ -88,4 +97,61 @@ suite("test_local_tvf_with_complex_type_insertinto_doris", "p0") { qt_sql """ drop table ${table_name} """ + // array-map + def table_arr = "comp_arr" + sql """ DROP TABLE IF EXISTS ${table_arr} """ + qt_sql """ + CREATE TABLE IF NOT EXISTS ${table_arr} ( + `id` int(11) NULL, + `aa` ARRAY> NULL, + `am` ARRAY> NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "is_being_synced" = "false", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); """ + + + qt_sql_orc_tvf """ + select * from local( + "file_path" = "${outFilePath}/comp_arr.orc", + "backend_id" = "${be_id}", + "format" = "orc");""" + + qt_sql """ + insert into ${table_arr} select * from local ( + "file_path" = "${outFilePath}/comp_arr.orc", + "backend_id" = "${be_id}", + "format" = "orc");""" + + qt_sql_orc_inner """ select * from ${table_arr} order by id; """ + + qt_sql """ truncate table ${table_arr} """ + + qt_sql_parquet_tvf """ + select * from local( + "file_path" = "${outFilePath}/comp_arr.parquet", + "backend_id" = "${be_id}", + "format" = "parquet"); """ + + qt_sql """ + insert into ${table_arr} select * from local( + "file_path" = "${outFilePath}/comp_arr.parquet", + "backend_id" = "${be_id}", + "format" = "parquet"); """ + + qt_sql_count """ select count(*) from ${table_arr} """ + + qt_sql_parquet_inner """ select * from ${table_arr} order by id""" + + qt_sql """ drop table ${table_arr} """ + + } diff --git a/regression-test/suites/external_table_p2/hive/test_external_catalog_hive.groovy b/regression-test/suites/external_table_p2/hive/test_external_catalog_hive.groovy index 7a9a80aff0183cf..09394a6122d5dbb 100644 --- a/regression-test/suites/external_table_p2/hive/test_external_catalog_hive.groovy +++ b/regression-test/suites/external_table_p2/hive/test_external_catalog_hive.groovy @@ -128,5 +128,20 @@ suite("test_external_catalog_hive", "p2,external,hive,external_remote,external_r logger.info("recoding select: " + res3.toString()) sql """alter catalog hms rename ${catalog_name};""" + + // test wrong access controller + test { + def tmp_name = "${catalog_name}" + "_wrong" + sql "drop catalog if exists ${tmp_name}" + sql """ + create catalog if not exists ${tmp_name} properties ( + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${extHiveHmsHost}:${extHiveHmsPort}', + 'access_controller.properties.ranger.service.name' = 'hive_wrong', + 'access_controller.class' = 'org.apache.doris.catalog.authorizer.RangerHiveAccessControllerFactory' + ); + """ + exception "Failed to init access controller: bound must be positive" + } } } diff --git a/regression-test/suites/external_table_p2/hive/test_hive_statistic.groovy b/regression-test/suites/external_table_p2/hive/test_hive_statistic.groovy index 2366267a27ea24b..c8163bba7e5aadd 100644 --- a/regression-test/suites/external_table_p2/hive/test_hive_statistic.groovy +++ b/regression-test/suites/external_table_p2/hive/test_hive_statistic.groovy @@ -30,6 +30,10 @@ suite("test_hive_statistic", "p2,external,hive,external_remote,external_remote_h ); """ logger.info("catalog " + catalog_name + " created") + + // Test analyze table without init. + sql """analyze table ${catalog_name}.tpch_1000_parquet.region with sync""" + sql """switch ${catalog_name};""" logger.info("switched to catalog " + catalog_name) sql """use statistics;""" @@ -243,6 +247,53 @@ suite("test_hive_statistic", "p2,external,hive,external_remote,external_remote_h sql """drop stats statistics""" result = sql """show column cached stats statistics""" assertTrue(result.size() == 0) + + sql """use multi_catalog""" + sql """analyze table logs1_parquet (log_time) with sync""" + def ctlId + def dbId + def tblId + result = sql """show proc '/catalogs'""" + + for (int i = 0; i < result.size(); i++) { + if (result[i][1] == catalog_name) { + ctlId = result[i][0] + } + } + result = sql """show proc '/catalogs/$ctlId'""" + for (int i = 0; i < result.size(); i++) { + if (result[i][1] == 'multi_catalog') { + dbId = result[i][0] + } + } + result = sql """show proc '/catalogs/$ctlId/$dbId'""" + for (int i = 0; i < result.size(); i++) { + if (result[i][1] == 'logs1_parquet') { + tblId = result[i][0] + } + } + + result = sql """select * from internal.__internal_schema.column_statistics where id = '${tblId}--1-log_time'""" + assertTrue(result.size() == 1) + def id = result[0][0] + def catalog_id = result[0][1] + def db_id = result[0][2] + def tbl_id = result[0][3] + def idx_id = result[0][4] + def col_id = result[0][5] + def count = result[0][7] + def ndv = result[0][8] + def null_count = result[0][9] + def data_size_in_bytes = result[0][12] + def update_time = result[0][13] + + sql """insert into internal.__internal_schema.column_statistics values ('$id', '$catalog_id', '$db_id', '$tbl_id', '$idx_id', '$col_id', NULL, $count, $ndv, $null_count, '', '', '$data_size_in_bytes', '$update_time')""" + + result = sql """show column stats logs1_parquet (log_time)""" + assertTrue(result.size() == 1) + assertTrue(result[0][6] == "N/A") + assertTrue(result[0][7] == "N/A") + sql """drop catalog ${catalog_name}""" } } diff --git a/regression-test/suites/javaudf_p0/test_javaudf_agg_map.groovy b/regression-test/suites/javaudf_p0/test_javaudf_agg_map.groovy index facd8fe1f9c3c5e..03f84e5b34ac7dc 100644 --- a/regression-test/suites/javaudf_p0/test_javaudf_agg_map.groovy +++ b/regression-test/suites/javaudf_p0/test_javaudf_agg_map.groovy @@ -27,9 +27,9 @@ suite("test_javaudf_agg_map") { try { try_sql("DROP FUNCTION IF EXISTS mapii(Map);") try_sql("DROP FUNCTION IF EXISTS mapid(Map);") - try_sql("DROP TABLE IF EXISTS db") + try_sql("DROP TABLE IF EXISTS db_agg_map") sql """ - CREATE TABLE IF NOT EXISTS db( + CREATE TABLE IF NOT EXISTS db_agg_map( `id` INT NULL COMMENT "", `i` INT NULL COMMENT "", `d` Double NULL COMMENT "", @@ -42,8 +42,8 @@ suite("test_javaudf_agg_map") { "replication_allocation" = "tag.location.default: 1", "storage_format" = "V2"); """ - sql """ INSERT INTO db VALUES(1, 10,1.1,{1:1,10:1,100:1},{1:1.1,11:11.1}); """ - sql """ INSERT INTO db VALUES(2, 20,2.2,{2:2,20:2,200:2},{2:2.2,22:22.2}); """ + sql """ INSERT INTO db_agg_map VALUES(1, 10,1.1,{1:1,10:1,100:1},{1:1.1,11:11.1}); """ + sql """ INSERT INTO db_agg_map VALUES(2, 20,2.2,{2:2,20:2,200:2},{2:2.2,22:22.2}); """ sql """ @@ -66,13 +66,13 @@ suite("test_javaudf_agg_map") { """ - qt_select_1 """ select mapid(mid) from db; """ + qt_select_1 """ select mapid(mid) from db_agg_map; """ - qt_select_2 """ select mapii(mii) from db; """ + qt_select_2 """ select mapii(mii) from db_agg_map; """ } finally { try_sql("DROP FUNCTION IF EXISTS mapii(Map);") try_sql("DROP FUNCTION IF EXISTS mapid(Map);") - try_sql("DROP TABLE IF EXISTS db") + try_sql("DROP TABLE IF EXISTS db_agg_map") } } diff --git a/regression-test/suites/javaudf_p0/test_javaudf_with_decimal.groovy b/regression-test/suites/javaudf_p0/test_javaudf_with_decimal.groovy new file mode 100644 index 000000000000000..bf2887283190b14 --- /dev/null +++ b/regression-test/suites/javaudf_p0/test_javaudf_with_decimal.groovy @@ -0,0 +1,97 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +import java.nio.charset.StandardCharsets +import java.nio.file.Files +import java.nio.file.Paths + +suite("test_javaudf_with_decimal") { + def jarPath = """${context.file.parent}/jars/java-udf-case-jar-with-dependencies.jar""" + log.info("Jar path: ${jarPath}".toString()) + try { + try_sql("drop function IF EXISTS getarrscale(Array);") + try_sql("drop function IF EXISTS getmapscale(Map);") + try_sql("drop function IF EXISTS retscale(int);") + try_sql("drop table IF EXISTS dbwithDecimal;") + sql """ + CREATE TABLE IF NOT EXISTS dbwithDecimal ( + `id` INT(11) NULL COMMENT "" , + `arr` Array NULL COMMENT "" , + `mp` Map NULL COMMENT "" + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "storage_format" = "V2" + ); + """ + sql """ INSERT INTO dbwithDecimal VALUES(1,[1.123,1.123456],{1.123:1.123456789}); """ + sql """ INSERT INTO dbwithDecimal VALUES(2,[2.123,2.123456],{2.123:2.123456789}); """ + + + sql """ + + CREATE FUNCTION getarrscale(Array) RETURNS int PROPERTIES ( + "file"="file://${jarPath}", + "symbol"="org.apache.doris.udf.MyArrayDecimal", + "always_nullable"="true", + "type"="JAVA_UDF" + ); + + """ + + sql """ + + CREATE FUNCTION getmapscale(Map) RETURNS int PROPERTIES ( + "file"="file://${jarPath}", + "symbol"="org.apache.doris.udf.MyMapDecimal", + "always_nullable"="true", + "type"="JAVA_UDF" + ); + + """ + + + sql """ + + CREATE FUNCTION retscale(int) RETURNS Map PROPERTIES ( + "file"="file://${jarPath}", + "symbol"="org.apache.doris.udf.MyMapRetDecimal", + "always_nullable"="true", + "type"="JAVA_UDF" + ); + """ + + sql """ + set enable_nereids_planner=false; + """ + + qt_select_1 """ select arr,getarrscale(arr) from dbwithDecimal order by id; """ + + qt_select_2 """ select mp,getmapscale(mp) from dbwithDecimal order by id ; """ + + qt_select_3 """ select id,retscale(id) from dbwithDecimal order by id; """ + } finally { + try_sql("drop function IF EXISTS getarrscale(Array);") + try_sql("drop function IF EXISTS getmapscale(Map);") + try_sql("drop function IF EXISTS retscale(int);") + try_sql("drop table IF EXISTS dbwithDecimal;") + } +} diff --git a/regression-test/suites/load_p0/http_stream/test_http_stream.groovy b/regression-test/suites/load_p0/http_stream/test_http_stream.groovy new file mode 100644 index 000000000000000..5d991c40c0dcb97 --- /dev/null +++ b/regression-test/suites/load_p0/http_stream/test_http_stream.groovy @@ -0,0 +1,832 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import java.util.Random; + +suite("test_http_stream", "p0") { + + // csv desc + // | c1 | c2 | c3 | c4 | c5 | c6 | c7 | c8 | + // | int | char | varchar| boolean | tinyint | smallint | bigint | largeint | + // | c9 | c10 | c11 | c12 | c13 | c14 | c15 | c16 | + // | float | double | decimal | decimalv3 | date | datev2 | datetime | datetimev2 | + + // 1. test column with currenttimestamp default value + def tableName1 = "test_http_stream_current_timestamp" + def db = "regression_test_load_p0_http_stream" + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName1} ( + id int, + name CHAR(10), + dt_1 DATETIME DEFAULT CURRENT_TIMESTAMP, + dt_2 DATETIMEV2 DEFAULT CURRENT_TIMESTAMP, + dt_3 DATETIMEV2(3) DEFAULT CURRENT_TIMESTAMP, + dt_4 DATETIMEV2(6) DEFAULT CURRENT_TIMESTAMP + ) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName1} (id, name) select c1, c2 from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql1 "select id, name from ${tableName1}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName1}" + } + + // 2. test change column order + def tableName2 = "test_http_stream_change_column_order" + + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName2} ( + k1 int, + k2 smallint NOT NULL, + k3 CHAR(10), + k4 bigint NOT NULL, + k5 decimal(6, 3) NOT NULL, + k6 float sum NOT NULL + ) + DISTRIBUTED BY HASH(k1) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName2} select c1, c6, c2, c7, c11, c9 from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql2 "select * from ${tableName2}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName2}" + } + + // 3. test with function + def tableName3 = "test_http_stream_function" + + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName3} ( + id int, + name CHAR(10), + year int, + month int, + day int + ) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName3} select c1, c2, year(c14), month(c14), day(c14) from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql3 "select * from ${tableName3}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName3}" + } + + // 4. test column number mismatch + def tableName4 = "test_http_stream_column_number_mismatch" + + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName4} ( + k1 int NOT NULL, + k2 CHAR(10) NOT NULL, + k3 smallint NOT NULL + ) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName4} select c1, c2, c6, c3 from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream.csv' + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("http_stream result: ${result}".toString()) + def json = parseJson(result) + assertEquals("fail", json.Status.toLowerCase()) + } + } + } catch (Exception e) { + assertTrue(e.getMessage().contains("Distribution column(id) doesn't exist"), e.getMessage()) + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName4}" + } + + // 5. test with default value + def tableName5 = "test_http_stream_default_value" + + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName5} ( + id int NOT NULL, + name CHAR(10) NOT NULL, + date DATE NOT NULL, + max_dwell_time INT DEFAULT "0", + min_dwell_time INT DEFAULT "99999" + ) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName5} (id, name, date) select c1, c2, c13 from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql5 "select * from ${tableName5}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName5}" + } + + // 6. test some column type + def tableName6 = "test_http_stream_column_type" + + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName6} ( + c_int int(11) NULL, + c_char char(15) NULL, + c_varchar varchar(100) NULL, + c_bool boolean NULL, + c_tinyint tinyint(4) NULL, + c_smallint smallint(6) NULL, + c_bigint bigint(20) NULL, + c_largeint largeint(40) NULL, + c_float float NULL, + c_double double NULL, + c_decimal decimal(6, 3) NULL, + c_decimalv3 decimalv3(6, 3) NULL, + c_date date NULL, + c_datev2 datev2 NULL, + c_datetime datetime NULL, + c_datetimev2 datetimev2(0) NULL + ) + DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName6} select * from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql6 "select * from ${tableName6}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName6}" + } + + // 7. test duplicate key + def tableName7 = "test_http_stream_duplicate_key" + + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName7} + ( + user_id LARGEINT NOT NULL, + username VARCHAR(50) NOT NULL, + city VARCHAR(20), + age SMALLINT, + sex TINYINT, + phone LARGEINT, + address VARCHAR(500), + register_time DATETIME + ) + DUPLICATE KEY(`user_id`, `username`) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName7} select * from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream_data_model.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql7 "select * from ${tableName7}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName7}" + } + + // 8. test merge on read unique key + def tableName8 = "test_http_stream_unique_key_merge_on_read" + + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName8} + ( + user_id LARGEINT NOT NULL, + username VARCHAR(50) NOT NULL, + city VARCHAR(20), + age SMALLINT, + sex TINYINT, + phone LARGEINT, + address VARCHAR(500), + register_time DATETIME + ) + UNIQUE KEY(`user_id`, `username`) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName8} select * from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream_data_model.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql8 "select * from ${tableName8}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName8}" + } + + // 9. test merge on write unique key + def tableName9 = "test_http_stream_unique_key_merge_on_write" + + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName9} + ( + user_id LARGEINT NOT NULL, + username VARCHAR(50) NOT NULL, + city VARCHAR(20), + age SMALLINT, + sex TINYINT, + phone LARGEINT, + address VARCHAR(500), + register_time DATETIME + ) + UNIQUE KEY(`user_id`, `username`) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "enable_unique_key_merge_on_write" = "true" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName9} select * from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream_data_model.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql9 "select * from ${tableName9}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName9}" + } + + // 10. test stream load multiple times + def tableName10 = "test_http_stream_multiple_times" + Random rd = new Random() + def disable_auto_compaction = "false" + if (rd.nextBoolean()) { + disable_auto_compaction = "true" + } + log.info("disable_auto_compaction: ${disable_auto_compaction}".toString()) + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName10} + ( + user_id LARGEINT NOT NULL, + username VARCHAR(50) NOT NULL, + money INT + ) + DUPLICATE KEY(`user_id`, `username`) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "disable_auto_compaction" = "${disable_auto_compaction}" + ) + """ + for (int i = 0; i < 3; ++i) { + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName10} select * from http_stream("format"="csv") + """ + time 10000 + file 'test_http_stream_multiple_times.csv' + 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(500, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + } + + qt_sql10 "select count(*) from ${tableName10}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName10}" + } + + // 11. test column separator + def tableName11 = "test_http_stream_column_separator" + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName11} ( + id int, + name CHAR(10), + dt_1 DATETIME DEFAULT CURRENT_TIMESTAMP, + dt_2 DATETIMEV2 DEFAULT CURRENT_TIMESTAMP, + dt_3 DATETIMEV2(3) DEFAULT CURRENT_TIMESTAMP, + dt_4 DATETIMEV2(6) DEFAULT CURRENT_TIMESTAMP + ) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName11} (id, name) select c1, c2 from http_stream("format"="csv", "column_separator"="--") + """ + time 10000 + file 'test_http_stream_column_separator.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql11 "select id, name from ${tableName11}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName11}" + } + + // 12. test line delimiter + def tableName12 = "test_http_stream_line_delimiter" + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName12} ( + id int, + name CHAR(10), + dt_1 DATETIME DEFAULT CURRENT_TIMESTAMP, + dt_2 DATETIMEV2 DEFAULT CURRENT_TIMESTAMP, + dt_3 DATETIMEV2(3) DEFAULT CURRENT_TIMESTAMP, + dt_4 DATETIMEV2(6) DEFAULT CURRENT_TIMESTAMP + ) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName12} (id, name) select c1, c2 from http_stream("format"="csv", "line_delimiter"="||") + """ + time 10000 + file 'test_http_stream_line_delimiter.csv' + 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(11, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql12 "select id, name from ${tableName12}" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName12}" + } + + // 13. test functions and aggregation operations + def tableName13 = "test_http_stream_agg" + try { + sql """ + CREATE TABLE IF NOT EXISTS ${tableName13} ( + name CHAR(10), + agg1 string + ) + DISTRIBUTED BY HASH(name) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + streamLoad { + set 'version', '1' + set 'sql', """ + insert into ${db}.${tableName13} (name, agg1) select c2, GROUP_CONCAT(c1, ' ') from http_stream("format"="csv") + group by c2 + """ + time 10000 + file 'test_http_stream_multiple_times.csv' + 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(443, json.NumberTotalRows) + assertEquals(0, json.NumberFilteredRows) + } + } + + qt_sql13 "select name, agg1 from ${tableName13} order by name" + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName13}" + } + + // 14. test parquet orc case + def tableName14 = "test_parquet_orc_case" + try { + sql """ DROP TABLE IF EXISTS ${tableName14} """ + sql """ + CREATE TABLE IF NOT EXISTS ${tableName14} ( + `WatchId` char(128), + `JavaEnable` smallint, + `Title` string, + `GoodEvent` smallint, + `EventTime` datetime, + `EventDate` date, + `CounterId` bigint, + `ClientIp` bigint, + `ClientIp6` char(50), + `RegionId` bigint, + `UserId` string, + `CounterClass` tinyint, + `Os` smallint, + `UserAgent` smallint, + `Url` string, + `Referer` string, + `Urldomain` string, + `RefererDomain` string, + `Refresh` smallint, + `IsRobot` smallint, + `RefererCategories` string, + `UrlCategories` string, + `UrlRegions` string, + `RefererRegions` string, + `ResolutionWidth` int, + `ResolutionHeight` int, + `ResolutionDepth` smallint, + `FlashMajor` smallint, + `FlashMinor` smallint, + `FlashMinor2` string, + `NetMajor` smallint, + `NetMinor` smallint, + `UserAgentMajor` int, + `UserAgentMinor` char(4), + `CookieEnable` smallint, + `JavascriptEnable` smallint, + `IsMobile` smallint, + `MobilePhone` smallint, + `MobilePhoneModel` string, + `Params` string, + `IpNetworkId` bigint, + `TraficSourceId` tinyint, + `SearchEngineId` int, + `SearchPhrase` string, + `AdvEngineId` smallint, + `IsArtifical` smallint, + `WindowClientWidth` int, + `WindowClientHeight` int, + `ClientTimeZone` smallint, + `ClientEventTime` datetime, + `SilverLightVersion1` smallint, + `SilverlightVersion2` smallint, + `SilverlightVersion3` bigint, + `SilverlightVersion4` int, + `PageCharset` string, + `CodeVersion` bigint, + `IsLink` smallint, + `IsDownload` smallint, + `IsNotBounce` smallint, + `FUniqId` string, + `Hid` bigint, + `IsOldCounter` smallint, + `IsEvent` smallint, + `IsParameter` smallint, + `DontCountHits` smallint, + `WithHash` smallint, + `HitColor` char(2), + `UtcEventTime` datetime, + `Age` smallint, + `Sex` smallint, + `Income` smallint, + `Interests` int, + `Robotness` smallint, + `GeneralInterests` string, + `RemoteIp` bigint, + `RemoteIp6` char(50), + `WindowName` int, + `OpenerName` int, + `historylength` smallint, + `BrowserLanguage` char(4), + `BrowserCountry` char(4), + `SocialNetwork` string, + `SocialAction` string, + `HttpError` int, + `SendTiming` int, + `DnsTiming` int, + `ConnectTiming` int, + `ResponseStartTiming` int, + `ResponseEndTiming` int, + `FetchTiming` int, + `RedirectTiming` int, + `DomInteractiveTiming` int, + `DomContentLoadedTiming` int, + `DomCompleteTiming` int, + `LoadEventStartTiming` int, + `LoadEventEndTiming` int, + `NsToDomContentLoadedTiming` int, + `FirstPaintTiming` int, + `RedirectCount` tinyint, + `SocialSourceNetworkId` smallint, + `SocialSourcePage` string, + `ParamPrice` bigint, + `ParamOrderId` string, + `ParamCurrency` char(6), + `ParamCurrencyId` int, + `GoalsReached` string, + `OpenStatServiceName` string, + `OpenStatCampaignId` string, + `OpenStatAdId` string, + `OpenStatSourceId` string, + `UtmSource` string, + `UtmMedium` string, + `UtmCampaign` string, + `UtmContent` string, + `UtmTerm` string, + `FromTag` string, + `HasGclId` smallint, + `RefererHash` string, + `UrlHash` string, + `ClId` bigint, + `YclId` string, + `ShareService` string, + `ShareUrl` string, + `ShareTitle` string, + `ParsedParamsKey1` string, + `ParsedParamsKey2` string, + `ParsedParamsKey3` string, + `ParsedParamsKey4` string, + `ParsedParamsKey5` string, + `ParsedParamsValueDouble` double, + `IsLandId` char(40), + `RequestNum` bigint, + `RequestTry` smallint + ) ENGINE=OLAP + DUPLICATE KEY(`WatchId`, `JavaEnable`) + DISTRIBUTED BY HASH(`WatchId`, `JavaEnable`) BUCKETS 3 + PROPERTIES ("replication_num" = "1"); + """ + + // streamLoad { + // set 'version', '1' + // set 'sql', """ + // insert into ${db}.${tableName14} select * from http_stream("format"="parquet") + // """ + // time 10000 + // set 'format', 'parquet' + // file 'test_http_stream_parquet_case.parquet' + // 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()) + // } + // } + // qt_sql13 "select * from ${tableName14} order by WatchId" + sql """truncate table ${tableName14}""" + + // streamLoad { + // set 'version', '1' + // set 'sql', """ + // insert into ${db}.${tableName14} select * from http_stream("format"="parquet") + // """ + // time 10000 + // set 'format', 'parquet' + // file 'test_http_stream_parquet_case.parquet' + // 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()) + // } + // } + // qt_sql13 "select * from ${tableName14} order by WatchId" + sql """truncate table ${tableName14}""" + + // streamLoad { + // set 'version', '1' + // set 'sql', """ + // insert into ${db}.${tableName14} select * from http_stream("format"="parquet") + // """ + // time 10000 + // set 'format', 'parquet' + // file 'test_http_stream_parquet_case.parquet' + // 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()) + // } + // } + // qt_sql13 "select * from ${tableName14} order by WatchId" + sql """truncate table ${tableName14}""" + + // streamLoad { + // set 'version', '1' + // set 'sql', """ + // insert into ${db}.${tableName14} select * from http_stream("format"="orc") + // """ + // time 10000 + // set 'format', 'orc' + // file 'test_http_stream_orc_case.orc' + // 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()) + // } + // } + // qt_sql13 "select * from ${tableName14} order by WatchId" + sql """truncate table ${tableName14}""" + + } finally { + try_sql "DROP TABLE IF EXISTS ${tableName14}" + } +} + diff --git a/regression-test/suites/load_p0/stream_load/test_stream_load_move_memtable.groovy b/regression-test/suites/load_p0/stream_load/test_stream_load_move_memtable.groovy index 08feba85cc6f5cf..f0910c2dee1b4fe 100644 --- a/regression-test/suites/load_p0/stream_load/test_stream_load_move_memtable.groovy +++ b/regression-test/suites/load_p0/stream_load/test_stream_load_move_memtable.groovy @@ -138,7 +138,7 @@ suite("test_stream_load_move_memtable", "p0") { assertEquals(3, rowCount[0][0]) // test load_nullable_to_not_nullable - def tableName2 = "load_nullable_to_not_nullable" + def tableName2 = "load_nullable_to_not_nullable_mm" sql """ DROP TABLE IF EXISTS ${tableName2} """ sql """ CREATE TABLE IF NOT EXISTS `${tableName2}` ( @@ -187,13 +187,13 @@ suite("test_stream_load_move_memtable", "p0") { order_qt_sql1 " SELECT * FROM ${tableName2}" // test common case - def tableName3 = "test_all" - def tableName4 = "test_less_col" - def tableName5 = "test_bitmap_and_hll" - def tableName6 = "test_unique_key" - def tableName7 = "test_unique_key_with_delete" - def tableName8 = "test_array" - def tableName10 = "test_struct" + def tableName3 = "test_all_mm" + def tableName4 = "test_less_col_mm" + def tableName5 = "test_bitmap_and_hll_mm" + def tableName6 = "test_unique_key_mm" + def tableName7 = "test_unique_key_with_delete_mm" + def tableName8 = "test_array_mm" + def tableName10 = "test_struct_mm" sql """ DROP TABLE IF EXISTS ${tableName3} """ sql """ DROP TABLE IF EXISTS ${tableName4} """ sql """ DROP TABLE IF EXISTS ${tableName5} """ @@ -803,7 +803,7 @@ suite("test_stream_load_move_memtable", "p0") { sql """sync""" // test immutable partition success - def tableName9 = "test_immutable_partition" + def tableName9 = "test_immutable_partition_mm" sql """ DROP TABLE IF EXISTS ${tableName9} """ sql """ CREATE TABLE IF NOT EXISTS ${tableName9} ( @@ -856,7 +856,7 @@ suite("test_stream_load_move_memtable", "p0") { order_qt_sql1 "select * from ${tableName9} order by k1, k2" // test common user - def tableName13 = "test_common_user" + def tableName13 = "test_common_user_mm" sql """ DROP TABLE IF EXISTS ${tableName13} """ sql """ CREATE TABLE IF NOT EXISTS ${tableName13} ( @@ -877,8 +877,8 @@ suite("test_stream_load_move_memtable", "p0") { PROPERTIES ("replication_allocation" = "tag.location.default: 1"); """ - sql """create USER common_user@'%' IDENTIFIED BY '123456'""" - sql """GRANT LOAD_PRIV ON *.* TO 'common_user'@'%';""" + sql """create USER common_user1@'%' IDENTIFIED BY '123456'""" + sql """GRANT LOAD_PRIV ON *.* TO 'common_user1'@'%';""" streamLoad { table "${tableName13}" @@ -906,10 +906,10 @@ suite("test_stream_load_move_memtable", "p0") { } sql "sync" - sql """DROP USER 'common_user'@'%'""" + sql """DROP USER 'common_user1'@'%'""" // test default value - def tableName14 = "test_default_value" + def tableName14 = "test_default_value_mm" sql """ DROP TABLE IF EXISTS ${tableName14} """ sql """ CREATE TABLE IF NOT EXISTS ${tableName14} ( diff --git a/regression-test/suites/load_p0/stream_load_with_sql/test_stream_load_with_sql.groovy b/regression-test/suites/load_p0/stream_load_with_sql/test_stream_load_with_sql.groovy deleted file mode 100644 index 8f0f207b6caa2e1..000000000000000 --- a/regression-test/suites/load_p0/stream_load_with_sql/test_stream_load_with_sql.groovy +++ /dev/null @@ -1,793 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -import java.util.Random; - -suite("test_stream_load_with_sql", "p0") { - - // csv desc - // | c1 | c2 | c3 | c4 | c5 | c6 | c7 | c8 | - // | int | char | varchar| boolean | tinyint | smallint | bigint | largeint | - // | c9 | c10 | c11 | c12 | c13 | c14 | c15 | c16 | - // | float | double | decimal | decimalv3 | date | datev2 | datetime | datetimev2 | - - // 1. test column with currenttimestamp default value - def tableName1 = "test_stream_load_with_sql_current_timestamp" - def db = "regression_test_load_p0_stream_load_with_sql" - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName1} ( - id int, - name CHAR(10), - dt_1 DATETIME DEFAULT CURRENT_TIMESTAMP, - dt_2 DATETIMEV2 DEFAULT CURRENT_TIMESTAMP, - dt_3 DATETIMEV2(3) DEFAULT CURRENT_TIMESTAMP, - dt_4 DATETIMEV2(6) DEFAULT CURRENT_TIMESTAMP - ) - DISTRIBUTED BY HASH(id) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName1} (id, name) select c1, c2 from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql1 "select id, name from ${tableName1}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName1}" - } - - // 2. test change column order - def tableName2 = "test_stream_load_with_sql_change_column_order" - - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName2} ( - k1 int, - k2 smallint NOT NULL, - k3 CHAR(10), - k4 bigint NOT NULL, - k5 decimal(6, 3) NOT NULL, - k6 float sum NOT NULL - ) - DISTRIBUTED BY HASH(k1) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName2} select c1, c6, c2, c7, c11, c9 from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql2 "select * from ${tableName2}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName2}" - } - - // 3. test with function - def tableName3 = "test_stream_load_with_sql_function" - - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName3} ( - id int, - name CHAR(10), - year int, - month int, - day int - ) - DISTRIBUTED BY HASH(id) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName3} select c1, c2, year(c14), month(c14), day(c14) from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql3 "select * from ${tableName3}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName3}" - } - - // 4. test column number mismatch - def tableName4 = "test_stream_load_with_sql_column_number_mismatch" - - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName4} ( - k1 int NOT NULL, - k2 CHAR(10) NOT NULL, - k3 smallint NOT NULL - ) - DISTRIBUTED BY HASH(id) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName4} select c1, c2, c6, c3 from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql.csv' - // 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()) - // } - // } - } catch (Exception e) { - assertTrue(e.getMessage().contains("Distribution column(id) doesn't exist"), e.getMessage()) - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName4}" - } - - // 5. test with default value - def tableName5 = "test_stream_load_with_sql_default_value" - - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName5} ( - id int NOT NULL, - name CHAR(10) NOT NULL, - date DATE NOT NULL, - max_dwell_time INT DEFAULT "0", - min_dwell_time INT DEFAULT "99999" - ) - DISTRIBUTED BY HASH(id) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName5} (id, name, date) select c1, c2, c13 from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql5 "select * from ${tableName5}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName5}" - } - - // 6. test some column type - def tableName6 = "test_stream_load_with_sql_column_type" - - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName6} ( - c_int int(11) NULL, - c_char char(15) NULL, - c_varchar varchar(100) NULL, - c_bool boolean NULL, - c_tinyint tinyint(4) NULL, - c_smallint smallint(6) NULL, - c_bigint bigint(20) NULL, - c_largeint largeint(40) NULL, - c_float float NULL, - c_double double NULL, - c_decimal decimal(6, 3) NULL, - c_decimalv3 decimalv3(6, 3) NULL, - c_date date NULL, - c_datev2 datev2 NULL, - c_datetime datetime NULL, - c_datetimev2 datetimev2(0) NULL - ) - DISTRIBUTED BY HASH(c_int) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName6} select * from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql6 "select * from ${tableName6}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName6}" - } - - // 7. test duplicate key - def tableName7 = "test_stream_load_with_sql_duplicate_key" - - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName7} - ( - user_id LARGEINT NOT NULL, - username VARCHAR(50) NOT NULL, - city VARCHAR(20), - age SMALLINT, - sex TINYINT, - phone LARGEINT, - address VARCHAR(500), - register_time DATETIME - ) - DUPLICATE KEY(`user_id`, `username`) - DISTRIBUTED BY HASH(`user_id`) BUCKETS 1 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName7} select * from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql_data_model.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql7 "select * from ${tableName7}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName7}" - } - - // 8. test merge on read unique key - def tableName8 = "test_stream_load_with_sql_unique_key_merge_on_read" - - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName8} - ( - user_id LARGEINT NOT NULL, - username VARCHAR(50) NOT NULL, - city VARCHAR(20), - age SMALLINT, - sex TINYINT, - phone LARGEINT, - address VARCHAR(500), - register_time DATETIME - ) - UNIQUE KEY(`user_id`, `username`) - DISTRIBUTED BY HASH(`user_id`) BUCKETS 1 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName8} select * from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql_data_model.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql8 "select * from ${tableName8}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName8}" - } - - // 9. test merge on write unique key - def tableName9 = "test_stream_load_with_sql_unique_key_merge_on_write" - - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName9} - ( - user_id LARGEINT NOT NULL, - username VARCHAR(50) NOT NULL, - city VARCHAR(20), - age SMALLINT, - sex TINYINT, - phone LARGEINT, - address VARCHAR(500), - register_time DATETIME - ) - UNIQUE KEY(`user_id`, `username`) - DISTRIBUTED BY HASH(`user_id`) BUCKETS 1 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1", - "enable_unique_key_merge_on_write" = "true" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName9} select * from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql_data_model.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql9 "select * from ${tableName9}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName9}" - } - - // 10. test stream load multiple times - def tableName10 = "test_stream_load_with_sql_multiple_times" - Random rd = new Random() - def disable_auto_compaction = "false" - if (rd.nextBoolean()) { - disable_auto_compaction = "true" - } - log.info("disable_auto_compaction: ${disable_auto_compaction}".toString()) - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName10} - ( - user_id LARGEINT NOT NULL, - username VARCHAR(50) NOT NULL, - money INT - ) - DUPLICATE KEY(`user_id`, `username`) - DISTRIBUTED BY HASH(`user_id`) BUCKETS 1 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1", - "disable_auto_compaction" = "${disable_auto_compaction}" - ) - """ - // for (int i = 0; i < 3; ++i) { - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName10} select * from stream("format"="csv") - // """ - // time 10000 - // file 'test_stream_load_with_sql_multiple_times.csv' - // 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(500, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - // } - - // qt_sql10 "select count(*) from ${tableName10}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName10}" - } - - // 11. test column separator - def tableName11 = "test_stream_load_with_sql_column_separator" - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName11} ( - id int, - name CHAR(10), - dt_1 DATETIME DEFAULT CURRENT_TIMESTAMP, - dt_2 DATETIMEV2 DEFAULT CURRENT_TIMESTAMP, - dt_3 DATETIMEV2(3) DEFAULT CURRENT_TIMESTAMP, - dt_4 DATETIMEV2(6) DEFAULT CURRENT_TIMESTAMP - ) - DISTRIBUTED BY HASH(id) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName11} (id, name) select c1, c2 from stream("format"="csv", "column_separator"="--") - // """ - // time 10000 - // file 'test_stream_load_with_sql_column_separator.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql11 "select id, name from ${tableName11}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName11}" - } - - // 12. test line delimiter - def tableName12 = "test_stream_load_with_sql_line_delimiter" - try { - sql """ - CREATE TABLE IF NOT EXISTS ${tableName12} ( - id int, - name CHAR(10), - dt_1 DATETIME DEFAULT CURRENT_TIMESTAMP, - dt_2 DATETIMEV2 DEFAULT CURRENT_TIMESTAMP, - dt_3 DATETIMEV2(3) DEFAULT CURRENT_TIMESTAMP, - dt_4 DATETIMEV2(6) DEFAULT CURRENT_TIMESTAMP - ) - DISTRIBUTED BY HASH(id) BUCKETS 1 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName12} (id, name) select c1, c2 from stream("format"="csv", "line_delimiter"="||") - // """ - // time 10000 - // file 'test_stream_load_with_sql_line_delimiter.csv' - // 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(11, json.NumberTotalRows) - // assertEquals(0, json.NumberFilteredRows) - // } - // } - - // qt_sql12 "select id, name from ${tableName12}" - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName12}" - } - - // 13. test parquet orc case - def tableName13 = "test_parquet_orc_case" - try { - sql """ DROP TABLE IF EXISTS ${tableName13} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName13} ( - `WatchId` char(128), - `JavaEnable` smallint, - `Title` string, - `GoodEvent` smallint, - `EventTime` datetime, - `EventDate` date, - `CounterId` bigint, - `ClientIp` bigint, - `ClientIp6` char(50), - `RegionId` bigint, - `UserId` string, - `CounterClass` tinyint, - `Os` smallint, - `UserAgent` smallint, - `Url` string, - `Referer` string, - `Urldomain` string, - `RefererDomain` string, - `Refresh` smallint, - `IsRobot` smallint, - `RefererCategories` string, - `UrlCategories` string, - `UrlRegions` string, - `RefererRegions` string, - `ResolutionWidth` int, - `ResolutionHeight` int, - `ResolutionDepth` smallint, - `FlashMajor` smallint, - `FlashMinor` smallint, - `FlashMinor2` string, - `NetMajor` smallint, - `NetMinor` smallint, - `UserAgentMajor` int, - `UserAgentMinor` char(4), - `CookieEnable` smallint, - `JavascriptEnable` smallint, - `IsMobile` smallint, - `MobilePhone` smallint, - `MobilePhoneModel` string, - `Params` string, - `IpNetworkId` bigint, - `TraficSourceId` tinyint, - `SearchEngineId` int, - `SearchPhrase` string, - `AdvEngineId` smallint, - `IsArtifical` smallint, - `WindowClientWidth` int, - `WindowClientHeight` int, - `ClientTimeZone` smallint, - `ClientEventTime` datetime, - `SilverLightVersion1` smallint, - `SilverlightVersion2` smallint, - `SilverlightVersion3` bigint, - `SilverlightVersion4` int, - `PageCharset` string, - `CodeVersion` bigint, - `IsLink` smallint, - `IsDownload` smallint, - `IsNotBounce` smallint, - `FUniqId` string, - `Hid` bigint, - `IsOldCounter` smallint, - `IsEvent` smallint, - `IsParameter` smallint, - `DontCountHits` smallint, - `WithHash` smallint, - `HitColor` char(2), - `UtcEventTime` datetime, - `Age` smallint, - `Sex` smallint, - `Income` smallint, - `Interests` int, - `Robotness` smallint, - `GeneralInterests` string, - `RemoteIp` bigint, - `RemoteIp6` char(50), - `WindowName` int, - `OpenerName` int, - `historylength` smallint, - `BrowserLanguage` char(4), - `BrowserCountry` char(4), - `SocialNetwork` string, - `SocialAction` string, - `HttpError` int, - `SendTiming` int, - `DnsTiming` int, - `ConnectTiming` int, - `ResponseStartTiming` int, - `ResponseEndTiming` int, - `FetchTiming` int, - `RedirectTiming` int, - `DomInteractiveTiming` int, - `DomContentLoadedTiming` int, - `DomCompleteTiming` int, - `LoadEventStartTiming` int, - `LoadEventEndTiming` int, - `NsToDomContentLoadedTiming` int, - `FirstPaintTiming` int, - `RedirectCount` tinyint, - `SocialSourceNetworkId` smallint, - `SocialSourcePage` string, - `ParamPrice` bigint, - `ParamOrderId` string, - `ParamCurrency` char(6), - `ParamCurrencyId` int, - `GoalsReached` string, - `OpenStatServiceName` string, - `OpenStatCampaignId` string, - `OpenStatAdId` string, - `OpenStatSourceId` string, - `UtmSource` string, - `UtmMedium` string, - `UtmCampaign` string, - `UtmContent` string, - `UtmTerm` string, - `FromTag` string, - `HasGclId` smallint, - `RefererHash` string, - `UrlHash` string, - `ClId` bigint, - `YclId` string, - `ShareService` string, - `ShareUrl` string, - `ShareTitle` string, - `ParsedParamsKey1` string, - `ParsedParamsKey2` string, - `ParsedParamsKey3` string, - `ParsedParamsKey4` string, - `ParsedParamsKey5` string, - `ParsedParamsValueDouble` double, - `IsLandId` char(40), - `RequestNum` bigint, - `RequestTry` smallint - ) ENGINE=OLAP - DUPLICATE KEY(`WatchId`, `JavaEnable`) - DISTRIBUTED BY HASH(`WatchId`, `JavaEnable`) BUCKETS 3 - PROPERTIES ("replication_num" = "1"); - """ - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName13} select * from stream("format"="parquet") - // """ - // time 10000 - // set 'format', 'parquet' - // file 'test_stream_load_with_sql_parquet_case.parquet' - // 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()) - // } - // } - // qt_sql13 "select * from ${tableName13} order by WatchId" - sql """truncate table ${tableName13}""" - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName13} select * from stream("format"="parquet") - // """ - // time 10000 - // set 'format', 'parquet' - // file 'test_stream_load_with_sql_parquet_case.parquet' - // 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()) - // } - // } - // qt_sql13 "select * from ${tableName13} order by WatchId" - sql """truncate table ${tableName13}""" - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName13} select * from stream("format"="parquet") - // """ - // time 10000 - // set 'format', 'parquet' - // file 'test_stream_load_with_sql_parquet_case.parquet' - // 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()) - // } - // } - // qt_sql13 "select * from ${tableName13} order by WatchId" - sql """truncate table ${tableName13}""" - - // streamLoad { - // set 'version', '1' - // set 'sql', """ - // insert into ${db}.${tableName13} select * from stream("format"="orc") - // """ - // time 10000 - // set 'format', 'orc' - // file 'test_stream_load_with_sql_orc_case.orc' - // 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()) - // } - // } - // qt_sql13 "select * from ${tableName13} order by WatchId" - sql """truncate table ${tableName13}""" - - } finally { - try_sql "DROP TABLE IF EXISTS ${tableName13}" - } -} - diff --git a/regression-test/suites/mv_p0/no_await/no_await.groovy b/regression-test/suites/mv_p0/no_await/no_await.groovy index 42f50095d555b1e..2daae738afcc1c6 100644 --- a/regression-test/suites/mv_p0/no_await/no_await.groovy +++ b/regression-test/suites/mv_p0/no_await/no_await.groovy @@ -19,27 +19,30 @@ import org.codehaus.groovy.runtime.IOGroovyMethods suite ("no_await") { + def tblName = "agg_have_dup_base_no_await" def waitDrop = { - def try_times = 100 + def try_times = 1000 def result = "null" + sql "sync;" while (!result.contains("FINISHED")) { - result = (sql "SHOW ALTER TABLE MATERIALIZED VIEW ORDER BY CreateTime DESC LIMIT 1;")[0] - Thread.sleep(500) + result = (sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tblName}' ORDER BY CreateTime DESC LIMIT 1;")[0] + Thread.sleep(1100) try_times -= 1 assertTrue(try_times > 0) } - - sql "drop materialized view k12s3m on agg_have_dup_base;" - while (!(sql "show create materialized view k12s3m on agg_have_dup_base;").empty) { + sql "sync;" + sql "drop materialized view k12s3m on ${tblName};" + while (!(sql "show create materialized view k12s3m on ${tblName};").empty) { sleep(100) try_times -= 1 assertTrue(try_times > 0) } + sql "sync;" } - sql 'drop table if exists agg_have_dup_base' - sql ''' - create table agg_have_dup_base ( + sql "drop table if exists ${tblName} force;" + sql """ + create table ${tblName} ( k1 int null, k2 int not null, k3 bigint null, @@ -48,95 +51,95 @@ suite ("no_await") { duplicate key (k1, k2, k3) distributed by hash(k1) buckets 3 properties("replication_num" = "1"); - ''' - sql "insert into agg_have_dup_base select e1, -4, -4, 'd' from (select 1 k1) as t lateral view explode_numbers(10000) tmp1 as e1;" + """ + sql "insert into ${tblName} select e1, -4, -4, 'd' from (select 1 k1) as t lateral view explode_numbers(10000) tmp1 as e1;" // do not await - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" waitDrop() - sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from agg_have_dup_base group by k1;" - sql 'insert into agg_have_dup_base select -4, -4, -4, \'d\'' - qt_mv 'select sum(k1) from agg_have_dup_base' + sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;" + sql "insert into ${tblName} select -4, -4, -4, \'d\'" + qt_mv "select sum(k1) from ${tblName}" } diff --git a/regression-test/suites/mv_p0/ssb/multiple_no_where/multiple_no_where.groovy b/regression-test/suites/mv_p0/ssb/multiple_no_where/multiple_no_where.groovy index b92b37c5e1ff6ec..e671f118233fce2 100644 --- a/regression-test/suites/mv_p0/ssb/multiple_no_where/multiple_no_where.groovy +++ b/regression-test/suites/mv_p0/ssb/multiple_no_where/multiple_no_where.groovy @@ -120,6 +120,8 @@ suite ("multiple_no_where") { FROM lineorder_flat GROUP BY YEAR, C_NATION,C_REGION,S_REGION,P_MFGR;""") + createMV ("""create materialized view temp_2 as SELECT lo_orderkey, sum(lo_extendedprice),max(lo_extendedprice), min(lo_extendedprice) from lineorder_flat group by lo_orderkey;""") + sql """INSERT INTO lineorder_flat (LO_ORDERDATE, LO_ORDERKEY, LO_LINENUMBER, LO_CUSTKEY, LO_PARTKEY, LO_SUPPKEY, LO_ORDERPRIORITY, LO_SHIPPRIORITY, LO_QUANTITY, LO_EXTENDEDPRICE, LO_ORDTOTALPRICE, LO_DISCOUNT, LO_REVENUE, LO_SUPPLYCOST, LO_TAX, LO_COMMITDATE, LO_SHIPMODE,C_NAME,C_ADDRESS,C_CITY,C_NATION,C_REGION,C_PHONE,C_MKTSEGMENT,S_NAME,S_ADDRESS,S_CITY,S_NATION,S_REGION,S_PHONE,P_NAME,P_MFGR,P_CATEGORY,P_BRAND,P_COLOR,P_TYPE,P_SIZE,P_CONTAINER) VALUES (19930101 , 2 , 2 , 2 , 2 , 2 ,'2',2 ,2 ,2 ,2 ,2 ,2 ,2 ,2 ,'2023-06-09','shipmode','name','address','city','nation','region','phone','mktsegment','name','address','city','nation','region','phone','name','mfgr','category','brand','color','type',4,'container');""" sql """INSERT INTO lineorder_flat (LO_ORDERDATE, LO_ORDERKEY, LO_LINENUMBER, LO_CUSTKEY, LO_PARTKEY, LO_SUPPKEY, LO_ORDERPRIORITY, LO_SHIPPRIORITY, LO_QUANTITY, LO_EXTENDEDPRICE, LO_ORDTOTALPRICE, LO_DISCOUNT, LO_REVENUE, LO_SUPPLYCOST, LO_TAX, LO_COMMITDATE, LO_SHIPMODE, C_NAME, C_ADDRESS, C_CITY, C_NATION, C_REGION, C_PHONE, C_MKTSEGMENT, S_NAME, S_ADDRESS, S_CITY, S_NATION, S_REGION, S_PHONE, P_NAME, P_MFGR, P_CATEGORY, P_BRAND, P_COLOR,P_TYPE,P_SIZE,P_CONTAINER) VALUES (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#12', 'MFGR#12', 'brand', 'color', 'type', 4 ,'container');""" @@ -223,4 +225,10 @@ suite ("multiple_no_where") { AND P_MFGR IN ('MFGR#1', 'MFGR#2') GROUP BY YEAR, C_NATION ORDER BY YEAR ASC, C_NATION ASC;""" + + explain { + sql("""SELECT lo_orderkey, sum(lo_extendedprice),max(lo_extendedprice), min(lo_extendedprice) from lineorder_flat group by lo_orderkey order by lo_orderkey;""") + contains "(temp_2)" + } + qt_select_temp_2 """SELECT lo_orderkey, sum(lo_extendedprice),max(lo_extendedprice), min(lo_extendedprice) from lineorder_flat group by lo_orderkey order by lo_orderkey;""" } diff --git a/regression-test/suites/mv_p0/test_dup_mv_div/test_dup_mv_div.groovy b/regression-test/suites/mv_p0/test_dup_mv_div/test_dup_mv_div.groovy new file mode 100644 index 000000000000000..17d2fbf2e5816e4 --- /dev/null +++ b/regression-test/suites/mv_p0/test_dup_mv_div/test_dup_mv_div.groovy @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_dup_mv_div") { + sql """set enable_nereids_planner=true""" + sql """SET enable_fallback_to_original_planner=false""" + sql """ DROP TABLE IF EXISTS d_table; """ + + sql """ + create table d_table( + k1 int null, + k2 int not null, + k3 bigint null, + k4 varchar(100) null + ) + duplicate key (k1,k2,k3) + distributed BY hash(k1) buckets 3 + properties("replication_num" = "1"); + """ + + sql "insert into d_table select 1,1,1,'1';" + sql "insert into d_table select 2,2,2,'b';" + sql "insert into d_table select 3,-3,null,'c';" + + createMV ("create materialized view kdiv as select k1,k2/1 from d_table;") + + sql "insert into d_table select -4,-4,-4,'4';" + + qt_select_star "select * from d_table order by k1;" + + explain { + sql("select k1,k2/1 from d_table order by k1;") + contains "(kdiv)" + } + qt_select_mv "select k1,k2/1 from d_table order by k1;" +} diff --git a/regression-test/suites/mv_p0/varchar_length/varchar_length.groovy b/regression-test/suites/mv_p0/varchar_length/varchar_length.groovy new file mode 100644 index 000000000000000..476758dec8296e9 --- /dev/null +++ b/regression-test/suites/mv_p0/varchar_length/varchar_length.groovy @@ -0,0 +1,41 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("varchar_length") { + + sql """DROP TABLE IF EXISTS test1; """ + + sql """ + CREATE TABLE test1( + vid VARCHAR(1) NOT NULL COMMENT "", + report_time int NOT NULL COMMENT '' + ) + ENGINE=OLAP + UNIQUE KEY(vid, report_time) + DISTRIBUTED BY HASH(vid) BUCKETS AUTO + PROPERTIES + ( + "replication_num" = "1" + ); + """ + + createMV ("CREATE MATERIALIZED VIEW mv_test as SELECT report_time, vid FROM test1 ORDER BY report_time DESC; ") + + qt_select_exp "desc test1 all" +} diff --git a/regression-test/suites/nereids_function_p0/load.groovy b/regression-test/suites/nereids_function_p0/load.groovy index 33ea1dd9f917500..2f7b61b6874fe7b 100644 --- a/regression-test/suites/nereids_function_p0/load.groovy +++ b/regression-test/suites/nereids_function_p0/load.groovy @@ -77,7 +77,35 @@ suite("load") { `z_lat` double null, `radius` double null, `linestring_wkt` varchar(50) null, - `polygon_wkt` varchar(50) null + `polygon_wkt` varchar(50) null, + `km_bool_tint` map null, + `km_tint_tint` map null, + `km_sint_tint` map null, + `km_int_tint` map null, + `km_bint_tint` map null, + `km_lint_tint` map null, + `km_float_tint` map null, + `km_dbl_tint` map null, + `km_dcml_tint` map null, + `km_chr_tint` map null, + `km_vchr_tint` map null, + `km_str_tint` map null, + `km_date_tint` map null, + `km_dtm_tint` map null, + `km_tint_bool` map null, + `km_int_int` map null, + `km_tint_sint` map null, + `km_tint_int` map null, + `km_tint_bint` map null, + `km_tint_lint` map null, + `km_tint_float` map null, + `km_tint_dbl` map null, + `km_tint_dcml` map null, + `km_tint_chr` map null, + `km_tint_vchr` map null, + `km_tint_str` map null, + `km_tint_date` map null, + `km_tint_dtm` map null ) engine=olap DISTRIBUTED BY HASH(`id`) BUCKETS 4 properties("replication_num" = "1") @@ -131,15 +159,43 @@ suite("load") { `kadcml` array not null, `st_point_str` string not null, `st_point_vc` varchar(50) not null, - `x_lng` double null, - `x_lat` double null, - `y_lng` double null, - `y_lat` double null, - `z_lng` double null, - `z_lat` double null, - `radius` double null, - `linestring_wkt` varchar(50) null, - `polygon_wkt` varchar(50) null + `x_lng` double not null, + `x_lat` double not null, + `y_lng` double not null, + `y_lat` double not null, + `z_lng` double not null, + `z_lat` double not null, + `radius` double not null, + `linestring_wkt` varchar(50) not null, + `polygon_wkt` varchar(50) not null, + `km_bool_tint` map not null, + `km_tint_tint` map not null, + `km_sint_tint` map not null, + `km_int_tint` map not null, + `km_bint_tint` map not null, + `km_lint_tint` map not null, + `km_float_tint` map not null, + `km_dbl_tint` map not null, + `km_dcml_tint` map not null, + `km_chr_tint` map not null, + `km_vchr_tint` map not null, + `km_str_tint` map not null, + `km_date_tint` map not null, + `km_dtm_tint` map not null, + `km_tint_bool` map not null, + `km_int_int` map not null, + `km_tint_sint` map not null, + `km_tint_int` map not null, + `km_tint_bint` map not null, + `km_tint_lint` map not null, + `km_tint_float` map not null, + `km_tint_dbl` map not null, + `km_tint_dcml` map not null, + `km_tint_chr` map not null, + `km_tint_vchr` map not null, + `km_tint_str` map not null, + `km_tint_date` map not null, + `km_tint_dtm` map not null ) engine=olap DISTRIBUTED BY HASH(`id`) BUCKETS 4 properties("replication_num" = "1") @@ -155,7 +211,11 @@ suite("load") { kdcmlv3s1, kdcmlv3s2, kdcmlv3s3, kchrs1, kchrs2, kchrs3, kvchrs1, kvchrs2, kvchrs3, kstr, kdt, kdtv2, kdtm, kdtmv2s1, kdtmv2s2, kdtmv2s3, kabool, katint, kasint, kaint, kabint, kalint, kafloat, kadbl, kadt, kadtm, kadtv2, kadtmv2, kachr, kavchr, kastr, kadcml, - st_point_str, st_point_vc, x_lng, x_lat, y_lng, y_lat, z_lng, z_lat, radius, linestring_wkt, polygon_wkt + st_point_str, st_point_vc, x_lng, x_lat, y_lng, y_lat, z_lng, z_lat, radius, linestring_wkt, polygon_wkt, + km_bool_tint, km_tint_tint, km_sint_tint, km_int_tint, km_bint_tint, km_lint_tint, km_float_tint, + km_dbl_tint, km_dcml_tint, km_chr_tint, km_vchr_tint, km_str_tint, km_date_tint, km_dtm_tint, + km_tint_bool, km_int_int, km_tint_sint, km_tint_int, km_tint_bint, km_tint_lint, km_tint_float, + km_tint_dbl, km_tint_dcml, km_tint_chr, km_tint_vchr, km_tint_str, km_tint_date, km_tint_dtm ''' file "fn_test.dat" } diff --git a/regression-test/suites/nereids_function_p0/scalar_function/C.groovy b/regression-test/suites/nereids_function_p0/scalar_function/C.groovy index 47ff0b9b5db499d..df896d9f29e1e0b 100644 --- a/regression-test/suites/nereids_function_p0/scalar_function/C.groovy +++ b/regression-test/suites/nereids_function_p0/scalar_function/C.groovy @@ -99,4 +99,8 @@ suite("nereids_scalar_fn_C") { qt_sql_cos_Double_notnull "select cos(kdbl) from fn_test_not_nullable order by kdbl" sql "select current_user() from fn_test" sql "select current_user() from fn_test_not_nullable" + + qt_char "select char(68, 111, 114, 105, 115), char(68, 111, 114, 105, 115 using utf8);" + qt_convert "select convert(1 using gbk), convert(1, string);" + } diff --git a/regression-test/suites/nereids_function_p0/scalar_function/E.groovy b/regression-test/suites/nereids_function_p0/scalar_function/E.groovy index e55df2d2b9c5c53..8962c1c795cfd4e 100644 --- a/regression-test/suites/nereids_function_p0/scalar_function/E.groovy +++ b/regression-test/suites/nereids_function_p0/scalar_function/E.groovy @@ -19,6 +19,10 @@ suite("nereids_scalar_fn_E") { sql 'use regression_test_nereids_function_p0' sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' + + sql """ + CREATE ENCRYPTKEY if not exists my_key AS "ABCD123456789"; + """ qt_sql_elt_Integer_Varchar "select elt(kint, kvchrs1) from fn_test order by kint, kvchrs1" qt_sql_elt_Integer_Varchar_notnull "select elt(kint, kvchrs1) from fn_test_not_nullable order by kint, kvchrs1" qt_sql_elt_Integer_String "select elt(kint, kstr) from fn_test order by kint, kstr" @@ -31,4 +35,6 @@ suite("nereids_scalar_fn_E") { qt_sql_exp_Double_notnull "select exp(kdbl) from fn_test_not_nullable order by kdbl" qt_sql_extract_url_parameter_Varchar_Varchar "select extract_url_parameter(kvchrs1, kvchrs1) from fn_test order by kvchrs1, kvchrs1" qt_sql_extract_url_parameter_Varchar_Varchar_notnull "select extract_url_parameter(kvchrs1, kvchrs1) from fn_test_not_nullable order by kvchrs1, kvchrs1" + + qt_sql_encryptkey "select key my_key, key regression_test_nereids_function_p0.my_key" } \ No newline at end of file diff --git a/regression-test/suites/nereids_function_p0/scalar_function/Map.groovy b/regression-test/suites/nereids_function_p0/scalar_function/Map.groovy new file mode 100644 index 000000000000000..6f3626ce2488ea3 --- /dev/null +++ b/regression-test/suites/nereids_function_p0/scalar_function/Map.groovy @@ -0,0 +1,232 @@ +// 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("nereids_scalar_fn_map") { + sql 'use regression_test_nereids_function_p0' + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + + // element_at + order_qt_element_at_bool_tint """ select km_bool_tint[kbool] from fn_test """ + order_qt_element_at_tint_tint """ select km_tint_tint[ktint] from fn_test """ + order_qt_element_at_sint_tint """ select km_sint_tint[ksint] from fn_test """ + order_qt_element_at_int_tint """ select km_int_tint[kint] from fn_test """ + order_qt_element_at_bint_tint """ select km_bint_tint[kbint] from fn_test """ + order_qt_element_at_lint_tint """ select km_lint_tint[klint] from fn_test """ + order_qt_element_at_float_tint """ select km_float_tint[kfloat] from fn_test """ + order_qt_element_at_dbl_tint """ select km_dbl_tint[kdbl] from fn_test """ + order_qt_element_at_dcml_tint """ select km_dcml_tint[kdcmls1] from fn_test """ + order_qt_element_at_chr_tint """ select km_chr_tint[kchrs1] from fn_test """ + order_qt_element_at_vchr_tint """ select km_vchr_tint[kvchrs1] from fn_test """ + order_qt_element_at_str_tint """ select km_str_tint[kstr] from fn_test """ + order_qt_element_at_date_tint """ select km_date_tint[kdt] from fn_test """ + order_qt_element_at_dtm_tint """ select km_dtm_tint[kdtm] from fn_test """ + order_qt_element_at_bool_tint_notnull """ select km_bool_tint[kbool] from fn_test_not_nullable """ + order_qt_element_at_tint_tint_notnull """ select km_tint_tint[ktint] from fn_test_not_nullable """ + order_qt_element_at_sint_tint_notnull """ select km_sint_tint[ksint] from fn_test_not_nullable """ + order_qt_element_at_int_tint_notnull """ select km_int_tint[kint] from fn_test_not_nullable """ + order_qt_element_at_bint_tint_notnull """ select km_bint_tint[kbint] from fn_test_not_nullable """ + order_qt_element_at_lint_tint_notnull """ select km_lint_tint[klint] from fn_test_not_nullable """ + order_qt_element_at_float_tint_notnull """ select km_float_tint[kfloat] from fn_test_not_nullable """ + order_qt_element_at_dbl_tint_notnull """ select km_dbl_tint[kdbl] from fn_test_not_nullable """ + order_qt_element_at_dcml_tint_notnull """ select km_dcml_tint[kdcmls1] from fn_test_not_nullable """ + order_qt_element_at_chr_tint_notnull """ select km_chr_tint[kchrs1] from fn_test_not_nullable """ + order_qt_element_at_vchr_tint_notnull """ select km_vchr_tint[kvchrs1] from fn_test_not_nullable """ + order_qt_element_at_str_tint_notnull """ select km_str_tint[kstr] from fn_test_not_nullable """ + order_qt_element_at_date_tint_notnull """ select km_date_tint[kdt] from fn_test_not_nullable """ + order_qt_element_at_dtm_tint_notnull """ select km_dtm_tint[kdtm] from fn_test_not_nullable """ + + // map size + order_qt_map_size_bool_tint """ select map_size(km_bool_tint) from fn_test """ + order_qt_map_size_tint_tint """ select map_size(km_tint_tint) from fn_test """ + order_qt_map_size_sint_tint """ select map_size(km_sint_tint) from fn_test """ + order_qt_map_size_int_tint """ select map_size(km_int_tint) from fn_test """ + order_qt_map_size_bint_tint """ select map_size(km_bint_tint) from fn_test """ + order_qt_map_size_lint_tint """ select map_size(km_lint_tint) from fn_test """ + order_qt_map_size_float_tint """ select map_size(km_float_tint) from fn_test """ + order_qt_map_size_dbl_tint """ select map_size(km_dbl_tint) from fn_test """ + order_qt_map_size_dcml_tint """ select map_size(km_dcml_tint) from fn_test """ + order_qt_map_size_chr_tint """ select map_size(km_chr_tint) from fn_test """ + order_qt_map_size_vchr_tint """ select map_size(km_vchr_tint) from fn_test """ + order_qt_map_size_str_tint """ select map_size(km_str_tint) from fn_test """ + order_qt_map_size_date_tint """ select map_size(km_date_tint) from fn_test """ + order_qt_map_size_dtm_tint """ select map_size(km_dtm_tint) from fn_test """ + order_qt_map_size_tint_bool """ select map_size(km_tint_bool) from fn_test """ + order_qt_map_size_int_int """ select map_size(km_int_int) from fn_test """ + order_qt_map_size_tint_sint """ select map_size(km_tint_sint) from fn_test """ + order_qt_map_size_tint_int """ select map_size(km_tint_int) from fn_test """ + order_qt_map_size_tint_bint """ select map_size(km_tint_bint) from fn_test """ + order_qt_map_size_tint_lint """ select map_size(km_tint_lint) from fn_test """ + order_qt_map_size_tint_float """ select map_size(km_tint_float) from fn_test """ + order_qt_map_size_tint_dbl """ select map_size(km_tint_dbl) from fn_test """ + order_qt_map_size_tint_dcml """ select map_size(km_tint_dcml) from fn_test """ + order_qt_map_size_tint_chr """ select map_size(km_tint_chr) from fn_test """ + order_qt_map_size_tint_vchr """ select map_size(km_tint_vchr) from fn_test """ + order_qt_map_size_tint_str """ select map_size(km_tint_str) from fn_test """ + order_qt_map_size_tint_date """ select map_size(km_tint_date) from fn_test """ + order_qt_map_size_tint_dtm """ select map_size(km_tint_dtm) from fn_test """ + order_qt_map_size_bool_tint_notnull """ select map_size(km_bool_tint) from fn_test_not_nullable """ + order_qt_map_size_tint_tint_notnull """ select map_size(km_tint_tint) from fn_test_not_nullable """ + order_qt_map_size_sint_tint_notnull """ select map_size(km_sint_tint) from fn_test_not_nullable """ + order_qt_map_size_int_tint_notnull """ select map_size(km_int_tint) from fn_test_not_nullable """ + order_qt_map_size_bint_tint_notnull """ select map_size(km_bint_tint) from fn_test_not_nullable """ + order_qt_map_size_lint_tint_notnull """ select map_size(km_lint_tint) from fn_test_not_nullable """ + order_qt_map_size_float_tint_notnull """ select map_size(km_float_tint) from fn_test_not_nullable """ + order_qt_map_size_dbl_tint_notnull """ select map_size(km_dbl_tint) from fn_test_not_nullable """ + order_qt_map_size_dcml_tint_notnull """ select map_size(km_dcml_tint) from fn_test_not_nullable """ + order_qt_map_size_chr_tint_notnull """ select map_size(km_chr_tint) from fn_test_not_nullable """ + order_qt_map_size_vchr_tint_notnull """ select map_size(km_vchr_tint) from fn_test_not_nullable """ + order_qt_map_size_str_tint_notnull """ select map_size(km_str_tint) from fn_test_not_nullable """ + order_qt_map_size_date_tint_notnull """ select map_size(km_date_tint) from fn_test_not_nullable """ + order_qt_map_size_dtm_tint_notnull """ select map_size(km_dtm_tint) from fn_test_not_nullable """ + order_qt_map_size_tint_bool_notnull """ select map_size(km_tint_bool) from fn_test_not_nullable """ + order_qt_map_size_int_int_notnull """ select map_size(km_int_int) from fn_test_not_nullable """ + order_qt_map_size_tint_sint_notnull """ select map_size(km_tint_sint) from fn_test_not_nullable """ + order_qt_map_size_tint_int_notnull """ select map_size(km_tint_int) from fn_test_not_nullable """ + order_qt_map_size_tint_bint_notnull """ select map_size(km_tint_bint) from fn_test_not_nullable """ + order_qt_map_size_tint_lint_notnull """ select map_size(km_tint_lint) from fn_test_not_nullable """ + order_qt_map_size_tint_float_notnull """ select map_size(km_tint_float) from fn_test_not_nullable """ + order_qt_map_size_tint_dbl_notnull """ select map_size(km_tint_dbl) from fn_test_not_nullable """ + order_qt_map_size_tint_dcml_notnull """ select map_size(km_tint_dcml) from fn_test_not_nullable """ + order_qt_map_size_tint_chr_notnull """ select map_size(km_tint_chr) from fn_test_not_nullable """ + order_qt_map_size_tint_vchr_notnull """ select map_size(km_tint_vchr) from fn_test_not_nullable """ + order_qt_map_size_tint_str_notnull """ select map_size(km_tint_str) from fn_test_not_nullable """ + order_qt_map_size_tint_date_notnull """ select map_size(km_tint_date) from fn_test_not_nullable """ + order_qt_map_size_tint_dtm_notnull """ select map_size(km_tint_dtm) from fn_test_not_nullable """ + + // map_keys + order_qt_map_keys_bool_tint """ select map_keys(km_bool_tint) from fn_test """ + order_qt_map_keys_tint_tint """ select map_keys(km_tint_tint) from fn_test """ + order_qt_map_keys_sint_tint """ select map_keys(km_sint_tint) from fn_test """ + order_qt_map_keys_int_tint """ select map_keys(km_int_tint) from fn_test """ + order_qt_map_keys_bint_tint """ select map_keys(km_bint_tint) from fn_test """ + order_qt_map_keys_lint_tint """ select map_keys(km_lint_tint) from fn_test """ + order_qt_map_keys_float_tint """ select map_keys(km_float_tint) from fn_test """ + order_qt_map_keys_dbl_tint """ select map_keys(km_dbl_tint) from fn_test """ + order_qt_map_keys_dcml_tint """ select map_keys(km_dcml_tint) from fn_test """ + order_qt_map_keys_chr_tint """ select map_keys(km_chr_tint) from fn_test """ + order_qt_map_keys_vchr_tint """ select map_keys(km_vchr_tint) from fn_test """ + order_qt_map_keys_str_tint """ select map_keys(km_str_tint) from fn_test """ + order_qt_map_keys_date_tint """ select map_keys(km_date_tint) from fn_test """ + order_qt_map_keys_dtm_tint """ select map_keys(km_dtm_tint) from fn_test """ + order_qt_map_keys_bool_tint_notnull """ select map_keys(km_bool_tint) from fn_test_not_nullable """ + order_qt_map_keys_tint_tint_notnull """ select map_keys(km_tint_tint) from fn_test_not_nullable """ + order_qt_map_keys_sint_tint_notnull """ select map_keys(km_sint_tint) from fn_test_not_nullable """ + order_qt_map_keys_int_tint_notnull """ select map_keys(km_int_tint) from fn_test_not_nullable """ + order_qt_map_keys_bint_tint_notnull """ select map_keys(km_bint_tint) from fn_test_not_nullable """ + order_qt_map_keys_lint_tint_notnull """ select map_keys(km_lint_tint) from fn_test_not_nullable """ + order_qt_map_keys_float_tint_notnull """ select map_keys(km_float_tint) from fn_test_not_nullable """ + order_qt_map_keys_dbl_tint_notnull """ select map_keys(km_dbl_tint) from fn_test_not_nullable """ + order_qt_map_keys_dcml_tint_notnull """ select map_keys(km_dcml_tint) from fn_test_not_nullable """ + order_qt_map_keys_chr_tint_notnull """ select map_keys(km_chr_tint) from fn_test_not_nullable """ + order_qt_map_keys_vchr_tint_notnull """ select map_keys(km_vchr_tint) from fn_test_not_nullable """ + order_qt_map_keys_str_tint_notnull """ select map_keys(km_str_tint) from fn_test_not_nullable """ + order_qt_map_keys_date_tint_notnull """ select map_keys(km_date_tint) from fn_test_not_nullable """ + order_qt_map_keys_dtm_tint_notnull """ select map_keys(km_dtm_tint) from fn_test_not_nullable """ + + // map_values + order_qt_map_values_tint_bool """ select map_values(km_tint_bool) from fn_test """ + order_qt_map_values_int_int """ select map_values(km_int_int) from fn_test """ + order_qt_map_values_tint_sint """ select map_values(km_tint_sint) from fn_test """ + order_qt_map_values_tint_int """ select map_values(km_tint_int) from fn_test """ + order_qt_map_values_tint_bint """ select map_values(km_tint_bint) from fn_test """ + order_qt_map_values_tint_lint """ select map_values(km_tint_lint) from fn_test """ + order_qt_map_values_tint_float """ select map_values(km_tint_float) from fn_test """ + order_qt_map_values_tint_dbl """ select map_values(km_tint_dbl) from fn_test """ + order_qt_map_values_tint_dcml """ select map_values(km_tint_dcml) from fn_test """ + order_qt_map_values_tint_chr """ select map_values(km_tint_chr) from fn_test """ + order_qt_map_values_tint_vchr """ select map_values(km_tint_vchr) from fn_test """ + order_qt_map_values_tint_str """ select map_values(km_tint_str) from fn_test """ + order_qt_map_values_tint_date """ select map_values(km_tint_date) from fn_test """ + order_qt_map_values_tint_dtm """ select map_values(km_tint_dtm) from fn_test """ + order_qt_map_values_tint_bool_notnull """ select map_values(km_tint_bool) from fn_test_not_nullable """ + order_qt_map_values_int_int_notnull """ select map_values(km_int_int) from fn_test_not_nullable """ + order_qt_map_values_tint_sint_notnull """ select map_values(km_tint_sint) from fn_test_not_nullable """ + order_qt_map_values_tint_int_notnull """ select map_values(km_tint_int) from fn_test_not_nullable """ + order_qt_map_values_tint_bint_notnull """ select map_values(km_tint_bint) from fn_test_not_nullable """ + order_qt_map_values_tint_lint_notnull """ select map_values(km_tint_lint) from fn_test_not_nullable """ + order_qt_map_values_tint_float_notnull """ select map_values(km_tint_float) from fn_test_not_nullable """ + order_qt_map_values_tint_dbl_notnull """ select map_values(km_tint_dbl) from fn_test_not_nullable """ + order_qt_map_values_tint_dcml_notnull """ select map_values(km_tint_dcml) from fn_test_not_nullable """ + order_qt_map_values_tint_chr_notnull """ select map_values(km_tint_chr) from fn_test_not_nullable """ + order_qt_map_values_tint_vchr_notnull """ select map_values(km_tint_vchr) from fn_test_not_nullable """ + order_qt_map_values_tint_str_notnull """ select map_values(km_tint_str) from fn_test_not_nullable """ + order_qt_map_values_tint_date_notnull """ select map_values(km_tint_date) from fn_test_not_nullable """ + order_qt_map_values_tint_dtm_notnull """ select map_values(km_tint_dtm) from fn_test_not_nullable """ + + // map_contains_key + order_qt_map_contains_key_bool_tint """ select map_contains_key(km_bool_tint, kbool) from fn_test """ + order_qt_map_contains_key_tint_tint """ select map_contains_key(km_tint_tint, ktint) from fn_test """ + order_qt_map_contains_key_sint_tint """ select map_contains_key(km_sint_tint, ksint) from fn_test """ + order_qt_map_contains_key_int_tint """ select map_contains_key(km_int_tint, kint) from fn_test """ + order_qt_map_contains_key_bint_tint """ select map_contains_key(km_bint_tint, kbint) from fn_test """ + order_qt_map_contains_key_lint_tint """ select map_contains_key(km_lint_tint, klint) from fn_test """ + order_qt_map_contains_key_float_tint """ select map_contains_key(km_float_tint, kfloat) from fn_test """ + order_qt_map_contains_key_dbl_tint """ select map_contains_key(km_dbl_tint, kdbl) from fn_test """ + order_qt_map_contains_key_dcml_tint """ select map_contains_key(km_dcml_tint, kdcmls1) from fn_test """ + order_qt_map_contains_key_chr_tint """ select map_contains_key(km_chr_tint, kchrs1) from fn_test """ + order_qt_map_contains_key_vchr_tint """ select map_contains_key(km_vchr_tint, kvchrs1) from fn_test """ + order_qt_map_contains_key_str_tint """ select map_contains_key(km_str_tint, kstr) from fn_test """ + order_qt_map_contains_key_date_tint """ select map_contains_key(km_date_tint, kdt) from fn_test """ + order_qt_map_contains_key_dtm_tint """ select map_contains_key(km_dtm_tint, kdtm) from fn_test """ + order_qt_map_contains_key_bool_tint_notnull """ select map_contains_key(km_bool_tint, kbool) from fn_test_not_nullable """ + order_qt_map_contains_key_tint_tint_notnull """ select map_contains_key(km_tint_tint, ktint) from fn_test_not_nullable """ + order_qt_map_contains_key_sint_tint_notnull """ select map_contains_key(km_sint_tint, ksint) from fn_test_not_nullable """ + order_qt_map_contains_key_int_tint_notnull """ select map_contains_key(km_int_tint, kint) from fn_test_not_nullable """ + order_qt_map_contains_key_bint_tint_notnull """ select map_contains_key(km_bint_tint, kbint) from fn_test_not_nullable """ + order_qt_map_contains_key_lint_tint_notnull """ select map_contains_key(km_lint_tint, klint) from fn_test_not_nullable """ + order_qt_map_contains_key_float_tint_notnull """ select map_contains_key(km_float_tint, kfloat) from fn_test_not_nullable """ + order_qt_map_contains_key_dbl_tint_notnull """ select map_contains_key(km_dbl_tint, kdbl) from fn_test_not_nullable """ + order_qt_map_contains_key_dcml_tint_notnull """ select map_contains_key(km_dcml_tint, kdcmls1) from fn_test_not_nullable """ + order_qt_map_contains_key_chr_tint_notnull """ select map_contains_key(km_chr_tint, kchrs1) from fn_test_not_nullable """ + order_qt_map_contains_key_vchr_tint_notnull """ select map_contains_key(km_vchr_tint, kvchrs1) from fn_test_not_nullable """ + order_qt_map_contains_key_str_tint_notnull """ select map_contains_key(km_str_tint, kstr) from fn_test_not_nullable """ + order_qt_map_contains_key_date_tint_notnull """ select map_contains_key(km_date_tint, kdt) from fn_test_not_nullable """ + order_qt_map_contains_key_dtm_tint_notnull """ select map_contains_key(km_dtm_tint, kdtm) from fn_test_not_nullable """ + + // map_contains_value + order_qt_map_contains_value_tint_bool """ select map_contains_value(km_tint_bool, kbool) from fn_test """ + order_qt_map_contains_value_int_int """ select map_contains_value(km_int_int, ktint) from fn_test """ + order_qt_map_contains_value_tint_sint """ select map_contains_value(km_tint_sint, ksint) from fn_test """ + order_qt_map_contains_value_tint_int """ select map_contains_value(km_tint_int, kint) from fn_test """ + order_qt_map_contains_value_tint_bint """ select map_contains_value(km_tint_bint, kbint) from fn_test """ + order_qt_map_contains_value_tint_lint """ select map_contains_value(km_tint_lint, klint) from fn_test """ + order_qt_map_contains_value_tint_float """ select map_contains_value(km_tint_float, kfloat) from fn_test """ + order_qt_map_contains_value_tint_dbl """ select map_contains_value(km_tint_dbl, kdbl) from fn_test """ + order_qt_map_contains_value_tint_dcml """ select map_contains_value(km_tint_dcml, kdcmls1) from fn_test """ + order_qt_map_contains_value_tint_chr """ select map_contains_value(km_tint_chr, kchrs1) from fn_test """ + order_qt_map_contains_value_tint_vchr """ select map_contains_value(km_tint_vchr, kvchrs1) from fn_test """ + order_qt_map_contains_value_tint_str """ select map_contains_value(km_tint_str, kstr) from fn_test """ + order_qt_map_contains_value_tint_date """ select map_contains_value(km_tint_date, kdt) from fn_test """ + order_qt_map_contains_value_tint_dtm """ select map_contains_value(km_tint_dtm, kdtm) from fn_test """ + order_qt_map_contains_value_tint_bool_notnull """ select map_contains_value(km_tint_bool, kbool) from fn_test_not_nullable """ + order_qt_map_contains_value_int_int_notnull """ select map_contains_value(km_int_int, ktint) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_sint_notnull """ select map_contains_value(km_tint_sint, ksint) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_int_notnull """ select map_contains_value(km_tint_int, kint) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_bint_notnull """ select map_contains_value(km_tint_bint, kbint) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_lint_notnull """ select map_contains_value(km_tint_lint, klint) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_float_notnull """ select map_contains_value(km_tint_float, kfloat) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_dbl_notnull """ select map_contains_value(km_tint_dbl, kdbl) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_dcml_notnull """ select map_contains_value(km_tint_dcml, kdcmls1) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_chr_notnull """ select map_contains_value(km_tint_chr, kchrs1) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_vchr_notnull """ select map_contains_value(km_tint_vchr, kvchrs1) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_str_notnull """ select map_contains_value(km_tint_str, kstr) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_date_notnull """ select map_contains_value(km_tint_date, kdt) from fn_test_not_nullable """ + order_qt_map_contains_value_tint_dtm_notnull """ select map_contains_value(km_tint_dtm, kdtm) from fn_test_not_nullable """ + + +} \ No newline at end of file diff --git a/regression-test/suites/nereids_p0/aggregate/aggregate.groovy b/regression-test/suites/nereids_p0/aggregate/aggregate.groovy index 7ac3fbe9c5cac74..f0ba091e0459119 100644 --- a/regression-test/suites/nereids_p0/aggregate/aggregate.groovy +++ b/regression-test/suites/nereids_p0/aggregate/aggregate.groovy @@ -314,4 +314,18 @@ suite("aggregate") { qt_aggregate """ select avg(distinct c_bigint), avg(distinct c_double) from regression_test_nereids_p0_aggregate.${tableName} """ qt_aggregate """ select count(distinct c_bigint),count(distinct c_double),count(distinct c_string),count(distinct c_date_1),count(distinct c_timestamp_1),count(distinct c_timestamp_2),count(distinct c_timestamp_3),count(distinct c_boolean) from regression_test_nereids_p0_aggregate.${tableName} """ qt_select_quantile_percent """ select QUANTILE_PERCENT(QUANTILE_UNION(TO_QUANTILE_STATE(c_bigint,2048)),0.5) from regression_test_nereids_p0_aggregate.${tableName}; """ + + sql "select k1 as k, k1 from tempbaseall group by k1 having k1 > 0" + sql "select k1 as k, k1 from tempbaseall group by k1 having k > 0" + + // remove distinct for max, min, any_value + def plan = sql( + """explain optimized plan SELECT max(distinct c_bigint), + min(distinct c_bigint), + any_value(distinct c_bigint) + FROM regression_test_nereids_p0_aggregate.${tableName};""" + ).toString() + assertTrue(plan.contains("max(c_bigint")) + assertTrue(plan.contains("min(c_bigint")) + assertTrue(plan.contains("any_value(c_bigint")) } diff --git a/regression-test/suites/nereids_p0/delete/delete_cte.groovy b/regression-test/suites/nereids_p0/delete/delete_cte.groovy new file mode 100644 index 000000000000000..bebbf035491cd90 --- /dev/null +++ b/regression-test/suites/nereids_p0/delete/delete_cte.groovy @@ -0,0 +1,108 @@ +// 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('nereids_delete_cte') { + def t1 = 't1_cte' + def t2 = 't2_cte' + def t3 = 't3_cte' + + sql "drop table if exists ${t1}" + sql """ + create table ${t1} ( + id int, + id1 int, + c1 bigint, + c2 string, + c3 double, + c4 date + ) unique key (id, id1) + distributed by hash(id, id1) + properties( + "replication_num"="1", + "function_column.sequence_col" = "c4", + "enable_unique_key_merge_on_write" = "true" + ); + """ + + sql "drop table if exists ${t2}" + sql """ + create table ${t2} ( + id int, + c1 bigint, + c2 string, + c3 double, + c4 date + ) unique key (id) + distributed by hash(id) + properties( + "replication_num"="1" + ); + """ + + sql "drop table if exists ${t3}" + sql """ + create table ${t3} ( + id int + ) distributed by hash(id) + properties( + "replication_num"="1" + ); + """ + + sql """ + INSERT INTO ${t1} VALUES + (1, 10, 1, '1', 1.0, '2000-01-01'), + (2, 20, 2, '2', 2.0, '2000-01-02'), + (3, 30, 3, '3', 3.0, '2000-01-03'); + """ + + sql """ + + INSERT INTO ${t2} VALUES + (1, 10, '10', 10.0, '2000-01-10'), + (2, 20, '20', 20.0, '2000-01-20'), + (3, 30, '30', 30.0, '2000-01-30'), + (4, 4, '4', 4.0, '2000-01-04'), + (5, 5, '5', 5.0, '2000-01-05'); + """ + + sql """ + INSERT INTO ${t3} VALUES + (1), + (4), + (5); + """ + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_nereids_dml=true" + + sql "insert into ${t1}(id, c1, c2, c3) select id, c1 * 2, c2, c3 from ${t1}" + sql "insert into ${t2}(id, c1, c2, c3) select id, c1, c2 * 2, c3 from ${t2}" + sql "insert into ${t2}(c1, c3) select c1 + 1, c3 + 1 from (select id, c1, c3 from ${t1} order by id, c1 limit 10) ${t1}, ${t3}" + + qt_sql "select * from ${t1} order by id, id1" + + sql """ + with cte as (select * from ${t3}) + delete from ${t1} + using ${t2} join cte on ${t2}.id = cte.id + where ${t1}.id = ${t2}.id; + """ + + qt_sql "select * from ${t1} order by id, id1" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_p0/hint/test_leading.groovy b/regression-test/suites/nereids_p0/hint/test_leading.groovy new file mode 100644 index 000000000000000..fa782ed06630ab2 --- /dev/null +++ b/regression-test/suites/nereids_p0/hint/test_leading.groovy @@ -0,0 +1,108 @@ +/* + * 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_leading") { + // create database and tables + sql 'DROP DATABASE IF EXISTS test_leading' + sql 'CREATE DATABASE IF NOT EXISTS test_leading' + sql 'use test_leading' + + // setting planner to nereids + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + + // create tables + sql """drop table if exists t1;""" + sql """drop table if exists t2;""" + sql """drop table if exists t3;""" + sql """drop table if exists t4;""" + + sql """create table t1 (c1 int, c11 int) distributed by hash(c1) buckets 3 properties('replication_num' = '1');""" + sql """create table t2 (c2 int, c22 int) distributed by hash(c2) buckets 3 properties('replication_num' = '1');""" + sql """create table t3 (c3 int, c33 int) distributed by hash(c3) buckets 3 properties('replication_num' = '1');""" + sql """create table t4 (c4 int, c44 int) distributed by hash(c4) buckets 3 properties('replication_num' = '1');""" + +//// test inner join with all edge and vertax is complete and equal predicates + qt_select1 """explain shape plan select /*+ leading(t2 t1) */ * from t1 join t2 on c1 = c2;""" + qt_select2 """explain shape plan select /*+ leading(t1 t2) */ * from t1 join t2 on c1 = c2;""" + qt_select3 """explain shape plan select /*+ leading(t1 t2 t3) */ * from t1 join t2 on c1 = c2 join t3 on c2 = c3;""" + qt_select4 """explain shape plan select /*+ leading(t1 {t2 t3}) */ * from t1 join t2 on c1 = c2 join t3 on c2 = c3;""" + qt_select5 """explain shape plan select /*+ leading(t1 {t2 t3} t4) */ * from t1 join t2 on c1 = c2 join t3 on c2 = c3 join t4 on c3 = c4;""" + qt_select6 """explain shape plan select /*+ leading({t1 t2} {t3 t4}) */ * from t1 join t2 on c1 = c2 join t3 on c2 = c3 join t4 on c3 = c4;""" + + // test inner join with part of edge and need cross join + qt_select7 """explain shape plan select /*+ leading({t1 t2} {t3 t4}) */ * from t1 join t2 on c1 = c2 join t3 on c2 = c3 join t4 on c3 = c4;""" + +//// test outer join which can swap + // (A leftjoin B on (Pab)) innerjoin C on (Pac) = (A innerjoin C on (Pac)) leftjoin B on (Pab) + qt_select8 """explain shape plan select * from t1 left join t2 on c1 = c2 join t3 on c1 = c3;""" + qt_select9 """explain shape plan select /*+ leading(t1 t3 t2) */ * from t1 left join t2 on c1 = c2 join t3 on c1 = c3;""" + + // (A leftjoin B on (Pab)) leftjoin C on (Pac) = (A leftjoin C on (Pac)) leftjoin B on (Pab) + qt_select10 """explain shape plan select * from t1 left join t2 on c1 = c2 left join t3 on c1 = c3;""" + qt_select11 """explain shape plan select /*+ leading(t1 t3 t2) */ * from t1 left join t2 on c1 = c2 left join t3 on c1 = c3;""" + + // (A leftjoin B on (Pab)) leftjoin C on (Pbc) = A leftjoin (B leftjoin C on (Pbc)) on (Pab) + qt_select12 """explain shape plan select /*+ leading(t1 {t2 t3}) */ * from t1 left join t2 on c1 = c2 left join t3 on c2 = c3;""" + +//// test outer join which can not swap + // A leftjoin (B join C on (Pbc)) on (Pab) != (A leftjoin B on (Pab)) join C on (Pbc) output should be unused when explain + // this can be done because left join can be eliminated to inner join + qt_select13 """explain shape plan select /*+ leading(t1 {t2 t3}) */ * from t1 left join t2 on c1 = c2 join t3 on c2 = c3;""" + + // this can not be done, expect not success but return right deep tree + qt_select14 """explain shape plan select * from t1 left join (select * from t2 join t3 on c2 = c3) as tmp on c1 = c2;""" + qt_select15 """explain shape plan select /*+ leading(t1 t2 t3) */ * from t1 left join (select * from t2 join t3 on c2 = c3) as tmp on c1 = c2;""" + +//// test semi join + qt_select16 """explain shape plan select * from t1 where c1 in (select c2 from t2);""" + qt_select17 """explain shape plan select /*+ leading(t2 t1) */ * from t1 where c1 in (select c2 from t2);""" + +//// test anti join + qt_select18 """explain shape plan select * from t1 where exists (select 1 from t2);""" + qt_select19 """explain shape plan select /*+ leading (t2 t1) */ * from t1 where exists (select 1 from t2);""" + +//// test cte + // inline cte, change join order of tables inside cte + qt_select20 """explain shape plan with cte as (select * from t1 join t2 on c1 = c2) select * from cte, t2;""" + qt_select21 """explain shape plan with cte as (select * from t1 join t2 on c1 = c2) select /*+ leading(t2 t1 t3) */ * from cte, t3;""" + // outside cte + // inside and outside together (after unnest subquery) + +//// test syntax error and unsupported feature + // not exist tables in leading: syntax error + qt_select22 """explain shape plan select /*+ leading(t66 t1) */ * from t1 join t2 on c1 = c2;""" + qt_select23 """explain shape plan select /*+ leading(t3 t1) */ * from t1 join t2 on c1 = c2;""" + // subquery alias as leading table + qt_select24 """explain shape plan with cte as (select * from t1 join t2 on c1 = c2) select /*+ leading(t2 cte t1) */ * from cte, t2;""" + // do not have all tables inside hint + qt_select25 """explain shape plan select /*+ leading(t1 t2) */ * from t1 join t2 on c1 = c2 join t3 on c2 = c3;""" + // duplicated table + qt_select26 """explain shape plan select /*+ leading(t1 t1 t2 t3) */ * from t1 join t2 on c1 = c2 join t3 on c2 = c3;""" + +//// test table alias + qt_select27 """explain shape plan select /*+ leading(t1 t_2) */ * from t1 join t2 t_2 on c1 = c2;""" + qt_select28 """explain shape plan select /*+ leading(t1 t2) */ * from t1 join t2 t_2 on c1 = c2;""" + qt_select29 """explain shape plan select /*+ leading(t1 t_1) */ * from t1 join t1 t_1 on t1.c1 = t_1.c1;""" + + sql """drop table if exists t1;""" + sql """drop table if exists t2;""" + sql """drop table if exists t3;""" + sql """drop table if exists t4;""" +} diff --git a/regression-test/suites/nereids_p0/insert_into_table/aggregate.groovy b/regression-test/suites/nereids_p0/insert_into_table/aggregate.groovy index 360acaaa545a25b..4c0b9924d6bf6a9 100644 --- a/regression-test/suites/nereids_p0/insert_into_table/aggregate.groovy +++ b/regression-test/suites/nereids_p0/insert_into_table/aggregate.groovy @@ -25,65 +25,65 @@ suite("nereids_insert_aggregate") { sql 'set enable_strict_consistency_dml=true' sql '''insert into nereids_insert_into_table_test.agg_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_11 'select * from agg_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_t with label label_agg_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_12 'select * from agg_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_t partition (p1, p2) with label label_agg - select * except(kaint, kmintint) from src where id < 4''' + select * except(kaint, kmintint, kjson) from src where id < 4''' sql 'sync' qt_13 'select * from agg_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_light_sc_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_21 'select * from agg_light_sc_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_light_sc_t with label label_agg_light_sc_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' qt_22 'select * from agg_light_sc_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_light_sc_t partition (p1, p2) with label label_agg_light_sc - select * except(kaint, kmintint) from src where id < 4''' + select * except(kaint, kmintint, kjson) from src where id < 4''' sql 'sync' qt_23 'select * from agg_light_sc_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_31 'select * from agg_not_null_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_not_null_t with label label_agg_not_null_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_32 'select * from agg_not_null_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_not_null_t partition (p1, p2) with label label_agg_not_null - select * except(kaint, kmintint) from src where id < 4 and id is not null''' + select * except(kaint, kmintint, kjson) from src where id < 4 and id is not null''' sql 'sync' qt_33 'select * from agg_not_null_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_light_sc_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_41 'select * from agg_light_sc_not_null_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_light_sc_not_null_t with label label_agg_light_sc_not_null_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_42 'select * from agg_light_sc_not_null_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_light_sc_not_null_t partition (p1, p2) with label label_agg_light_sc_not_null - select * except(kaint, kmintint) from src where id < 4 and id is not null''' + select * except(kaint, kmintint, kjson) from src where id < 4 and id is not null''' sql 'sync' qt_43 'select * from agg_light_sc_not_null_t order by id, kint' @@ -92,12 +92,12 @@ suite("nereids_insert_aggregate") { sql 'alter table agg_light_sc_not_null_t rename column ktint ktinyint' sql '''insert into nereids_insert_into_table_test.agg_light_sc_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_lsc1 'select * from agg_light_sc_t order by id, kint' sql '''insert into nereids_insert_into_table_test.agg_light_sc_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_lsc2 'select * from agg_light_sc_not_null_t order by id, kint' diff --git a/regression-test/suites/nereids_p0/insert_into_table/ddl/json_t.sql b/regression-test/suites/nereids_p0/insert_into_table/ddl/json_t.sql new file mode 100644 index 000000000000000..d3c4c0a5e120e1d --- /dev/null +++ b/regression-test/suites/nereids_p0/insert_into_table/ddl/json_t.sql @@ -0,0 +1,10 @@ +create table json_t ( + `id` int null, + `kjson` json null +) +engine=OLAP +duplicate key(id) +distributed by hash(id) buckets 4 +properties ( + "replication_num"="1" +); \ No newline at end of file diff --git a/regression-test/suites/nereids_p0/insert_into_table/duplicate.groovy b/regression-test/suites/nereids_p0/insert_into_table/duplicate.groovy index 546f16e2160b4ec..633004d71be8f7b 100644 --- a/regression-test/suites/nereids_p0/insert_into_table/duplicate.groovy +++ b/regression-test/suites/nereids_p0/insert_into_table/duplicate.groovy @@ -25,66 +25,66 @@ suite("nereids_insert_duplicate") { sql 'set enable_strict_consistency_dml=true' sql '''insert into dup_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_11 'select * from dup_t order by id, kint' sql '''insert into dup_t with label label_dup_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_12 'select * from dup_t order by id, kint' sql '''insert into dup_t partition (p1, p2) with label label_dup - select * except(kaint, kmintint) from src where id < 4''' + select * except(kaint, kmintint, kjson) from src where id < 4''' sql 'sync' qt_13 'select * from dup_t order by id, kint' sql '''insert into dup_light_sc_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_21 'select * from dup_light_sc_t order by id, kint' sql '''insert into dup_light_sc_t with label label_dup_light_sc_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_22 'select * from dup_light_sc_t order by id, kint' sql '''insert into dup_light_sc_t partition (p1, p2) with label label_dup_light_sc - select * except(kaint, kmintint) from src where id < 4''' + select * except(kaint, kmintint, kjson) from src where id < 4''' sql 'sync' qt_23 'select * from dup_light_sc_t order by id, kint' sql '''insert into dup_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_31 'select * from dup_not_null_t order by id, kint' sql '''insert into dup_not_null_t with label label_dup_not_null_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_32 'select * from dup_not_null_t order by id, kint' sql '''insert into dup_not_null_t partition (p1, p2) with label label_dup_not_null - select * except(kaint, kmintint) from src where id < 4 and id is not null''' + select * except(kaint, kmintint, kjson) from src where id < 4 and id is not null''' sql 'sync' qt_33 'select * from dup_not_null_t order by id, kint' sql '''insert into dup_light_sc_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_41 'select * from dup_light_sc_not_null_t order by id, kint' sql '''insert into dup_light_sc_not_null_t with label label_dup_light_sc_not_null_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_42 'select * from dup_light_sc_not_null_t order by id, kint' sql '''insert into dup_light_sc_not_null_t partition (p1, p2) with label label_dup_light_sc_not_null - select * except(kaint, kmintint) from src where id < 4 and id is not null''' + select * except(kaint, kmintint, kjson) from src where id < 4 and id is not null''' sql 'sync' qt_43 'select * from dup_light_sc_not_null_t order by id, kint' @@ -93,12 +93,12 @@ suite("nereids_insert_duplicate") { sql 'alter table dup_light_sc_not_null_t rename column ktint ktinyint' sql '''insert into dup_light_sc_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_lsc1 'select * from dup_light_sc_t order by id, kint' sql '''insert into dup_light_sc_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_lsc2 'select * from dup_light_sc_not_null_t order by id, kint' diff --git a/regression-test/suites/nereids_p0/insert_into_table/load.groovy b/regression-test/suites/nereids_p0/insert_into_table/load.groovy index 0b2f6fc213d697d..c8b9645a5f5d167 100644 --- a/regression-test/suites/nereids_p0/insert_into_table/load.groovy +++ b/regression-test/suites/nereids_p0/insert_into_table/load.groovy @@ -43,7 +43,8 @@ suite("load") { `kdcml64v3` decimalv3(10, 5) null, `kdcml128v3` decimalv3(20, 8) null, `kaint` array null, - `kmintint` map null + `kmintint` map null, + `kjson` json null ) engine=OLAP duplicate key(id) distributed by hash(id) buckets 4 @@ -67,7 +68,7 @@ suite("load") { 'agg_nop_t', 'agg_t', 'agg_type_cast', 'dup_nop_t', 'dup_t', 'dup_type_cast', 'uni_nop_t', 'uni_t', 'uni_type_cast', - 'map_t', 'random_t' + 'map_t', 'random_t', 'json_t' ] for (String file in files) { diff --git a/regression-test/suites/nereids_p0/insert_into_table/no_partition.groovy b/regression-test/suites/nereids_p0/insert_into_table/no_partition.groovy index 666188d618419de..08d2f4b67bd58c0 100644 --- a/regression-test/suites/nereids_p0/insert_into_table/no_partition.groovy +++ b/regression-test/suites/nereids_p0/insert_into_table/no_partition.groovy @@ -28,7 +28,7 @@ suite('nereids_insert_no_partition') { // TODO: test turn off pipeline when dml, remove it if pipeline sink is ok sql ''' insert into uni_light_sc_mow_not_null_nop_t with t as( - select * except(kaint, kmintint) from src where id is not null) + select * except(kaint, kmintint, kjson) from src where id is not null) select * from t left semi join t t2 on t.id = t2.id; ''' @@ -36,176 +36,176 @@ suite('nereids_insert_no_partition') { } sql '''insert into agg_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_11 'select * from agg_nop_t order by id, kint' sql '''insert into agg_nop_t with label label_agg_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_12 'select * from agg_nop_t order by id, kint' sql '''insert into agg_light_sc_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_21 'select * from agg_light_sc_nop_t order by id, kint' sql '''insert into agg_light_sc_nop_t with label label_agg_light_sc_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_22 'select * from agg_light_sc_nop_t order by id, kint' sql '''insert into agg_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_31 'select * from agg_not_null_nop_t order by id, kint' sql '''insert into agg_not_null_nop_t with label label_agg_not_null_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_32 'select * from agg_not_null_nop_t order by id, kint' sql '''insert into agg_light_sc_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_41 'select * from agg_light_sc_not_null_nop_t order by id, kint' sql '''insert into agg_light_sc_not_null_nop_t with label label_agg_light_sc_not_null_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_42 'select * from agg_light_sc_not_null_nop_t order by id, kint' sql '''insert into dup_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_11 'select * from dup_nop_t order by id, kint' sql '''insert into dup_nop_t with label label_dup_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_12 'select * from dup_nop_t order by id, kint' sql '''insert into dup_light_sc_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_21 'select * from dup_light_sc_nop_t order by id, kint' sql '''insert into dup_light_sc_nop_t with label label_dup_light_sc_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_22 'select * from dup_light_sc_nop_t order by id, kint' sql '''insert into dup_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_31 'select * from dup_not_null_nop_t order by id, kint' sql '''insert into dup_not_null_nop_t with label label_dup_not_null_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_32 'select * from dup_not_null_nop_t order by id, kint' sql '''insert into dup_light_sc_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_41 'select * from dup_light_sc_not_null_nop_t order by id, kint' sql '''insert into dup_light_sc_not_null_nop_t with label label_dup_light_sc_not_null_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_42 'select * from dup_light_sc_not_null_nop_t order by id, kint' sql '''insert into uni_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_11 'select * from uni_nop_t order by id, kint' sql '''insert into uni_nop_t with label label_uni_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_12 'select * from uni_nop_t order by id, kint' sql '''insert into uni_light_sc_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' qt_21 'select * from uni_light_sc_nop_t order by id, kint' sql '''insert into uni_light_sc_nop_t with label label_uni_light_sc_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_22 'select * from uni_light_sc_nop_t order by id, kint' sql '''insert into uni_mow_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_31 'select * from uni_mow_nop_t order by id, kint' sql '''insert into uni_mow_nop_t with label label_uni_mow_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_32 'select * from uni_mow_nop_t order by id, kint' sql '''insert into uni_light_sc_mow_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_41 'select * from uni_light_sc_mow_nop_t order by id, kint' sql '''insert into uni_light_sc_mow_nop_t with label label_uni_light_sc_mow_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_42 'select * from uni_light_sc_mow_nop_t order by id, kint' sql '''insert into uni_mow_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_51 'select * from uni_mow_not_null_nop_t order by id, kint' sql '''insert into uni_mow_not_null_nop_t with label label_uni_not_null_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_52 'select * from uni_mow_not_null_nop_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_61 'select * from uni_light_sc_mow_not_null_nop_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_nop_t with label label_uni_light_sc_not_null_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_62 'select * from uni_light_sc_mow_not_null_nop_t order by id, kint' sql '''insert into uni_mow_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_71 'select * from uni_mow_not_null_nop_t order by id, kint' sql '''insert into uni_mow_not_null_nop_t with label label_uni_mow_not_null_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_72 'select * from uni_mow_not_null_nop_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_81 'select * from uni_light_sc_mow_not_null_nop_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_nop_t with label label_uni_light_sc_mow_not_null_cte_nop - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_82 'select * from uni_light_sc_mow_not_null_nop_t order by id, kint' @@ -215,12 +215,12 @@ suite('nereids_insert_no_partition') { sql 'alter table agg_light_sc_not_null_nop_t rename column ktint ktinyint' sql '''insert into agg_light_sc_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_lsc1 'select * from agg_light_sc_nop_t order by id, kint' sql '''insert into agg_light_sc_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_lsc2 'select * from agg_light_sc_not_null_nop_t order by id, kint' @@ -229,12 +229,12 @@ suite('nereids_insert_no_partition') { sql 'alter table dup_light_sc_not_null_nop_t rename column ktint ktinyint' sql '''insert into dup_light_sc_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_lsc1 'select * from dup_light_sc_nop_t order by id, kint' sql '''insert into dup_light_sc_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_lsc2 'select * from dup_light_sc_not_null_nop_t order by id, kint' @@ -245,22 +245,22 @@ suite('nereids_insert_no_partition') { sql 'alter table uni_light_sc_mow_not_null_nop_t rename column ktint ktinyint' sql '''insert into uni_light_sc_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_lsc1 'select * from uni_light_sc_nop_t order by id, kint' sql '''insert into uni_light_sc_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_lsc2 'select * from uni_light_sc_not_null_nop_t order by id, kint' sql '''insert into uni_light_sc_mow_nop_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_lsc3 'select * from uni_light_sc_mow_nop_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_nop_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_lsc4 'select * from uni_light_sc_mow_not_null_nop_t order by id, kint' diff --git a/regression-test/suites/nereids_p0/insert_into_table/unique.groovy b/regression-test/suites/nereids_p0/insert_into_table/unique.groovy index bdad2aa902ed614..dc9adf03cd55621 100644 --- a/regression-test/suites/nereids_p0/insert_into_table/unique.groovy +++ b/regression-test/suites/nereids_p0/insert_into_table/unique.groovy @@ -25,130 +25,130 @@ suite("nereids_insert_unique") { sql 'set enable_strict_consistency_dml=true' sql '''insert into uni_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_11 'select * from uni_t order by id, kint' sql '''insert into uni_t with label label_uni_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_12 'select * from uni_t order by id, kint' sql '''insert into uni_t partition (p1, p2) with label label_uni - select * except(kaint, kmintint) from src where id < 4''' + select * except(kaint, kmintint, kjson) from src where id < 4''' sql 'sync' qt_13 'select * from uni_t order by id, kint' sql '''insert into uni_light_sc_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_21 'select * from uni_light_sc_t order by id, kint' sql '''insert into uni_light_sc_t with label label_uni_light_sc_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_22 'select * from uni_light_sc_t order by id, kint' sql '''insert into uni_light_sc_t partition (p1, p2) with label label_uni_light_sc - select * except(kaint, kmintint) from src where id < 4''' + select * except(kaint, kmintint, kjson) from src where id < 4''' sql 'sync' qt_23 'select * from uni_light_sc_t order by id, kint' sql '''insert into uni_mow_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_31 'select * from uni_mow_t order by id, kint' sql '''insert into uni_mow_t with label label_uni_mow_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_32 'select * from uni_mow_t order by id, kint' sql '''insert into uni_mow_t partition (p1, p2) with label label_uni_mow - select * except(kaint, kmintint) from src where id < 4''' + select * except(kaint, kmintint, kjson) from src where id < 4''' sql 'sync' qt_33 'select * from uni_mow_t order by id, kint' sql '''insert into uni_light_sc_mow_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_41 'select * from uni_light_sc_mow_t order by id, kint' sql '''insert into uni_light_sc_mow_t with label label_uni_light_sc_mow_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte''' sql 'sync' qt_42 'select * from uni_light_sc_mow_t order by id, kint' sql '''insert into uni_light_sc_mow_t partition (p1, p2) with label label_uni_light_sc_mow - select * except(kaint, kmintint) from src where id < 4''' + select * except(kaint, kmintint, kjson) from src where id < 4''' sql 'sync' qt_43 'select * from uni_light_sc_mow_t order by id, kint' sql '''insert into uni_mow_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_51 'select * from uni_mow_not_null_t order by id, kint' sql '''insert into uni_mow_not_null_t with label label_uni_not_null_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_52 'select * from uni_mow_not_null_t order by id, kint' sql '''insert into uni_mow_not_null_t partition (p1, p2) with label label_uni_not_null - select * except(kaint, kmintint) from src where id < 4 and id is not null''' + select * except(kaint, kmintint, kjson) from src where id < 4 and id is not null''' sql 'sync' qt_53 'select * from uni_mow_not_null_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_61 'select * from uni_light_sc_mow_not_null_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_t with label label_uni_light_sc_not_null_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_62 'select * from uni_light_sc_mow_not_null_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_t partition (p1, p2) with label label_uni_light_sc_not_null - select * except(kaint, kmintint) from src where id < 4 and id is not null''' + select * except(kaint, kmintint, kjson) from src where id < 4 and id is not null''' sql 'sync' qt_63 'select * from uni_light_sc_mow_not_null_t order by id, kint' sql '''insert into uni_mow_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_71 'select * from uni_mow_not_null_t order by id, kint' sql '''insert into uni_mow_not_null_t with label label_uni_mow_not_null_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_72 'select * from uni_mow_not_null_t order by id, kint' sql '''insert into uni_mow_not_null_t partition (p1, p2) with label label_uni_mow_not_null - select * except(kaint, kmintint) from src where id < 4 and id is not null''' + select * except(kaint, kmintint, kjson) from src where id < 4 and id is not null''' sql 'sync' qt_73 'select * from uni_mow_not_null_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_81 'select * from uni_light_sc_mow_not_null_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_t with label label_uni_light_sc_mow_not_null_cte - with cte as (select * except(kaint, kmintint) from src) + with cte as (select * except(kaint, kmintint, kjson) from src) select * from cte where id is not null''' sql 'sync' qt_82 'select * from uni_light_sc_mow_not_null_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_t partition (p1, p2) with label label_uni_light_sc_mow_not_null - select * except(kaint, kmintint) from src where id < 4 and id is not null''' + select * except(kaint, kmintint, kjson) from src where id < 4 and id is not null''' sql 'sync' qt_83 'select * from uni_light_sc_mow_not_null_t order by id, kint' @@ -159,22 +159,22 @@ suite("nereids_insert_unique") { sql 'alter table uni_light_sc_mow_not_null_t rename column ktint ktinyint' sql '''insert into uni_light_sc_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_lsc1 'select * from uni_light_sc_t order by id, kint' sql '''insert into uni_light_sc_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_lsc2 'select * from uni_light_sc_not_null_t order by id, kint' sql '''insert into uni_light_sc_mow_t - select * except(kaint, kmintint) from src''' + select * except(kaint, kmintint, kjson) from src''' sql 'sync' qt_lsc3 'select * from uni_light_sc_mow_t order by id, kint' sql '''insert into uni_light_sc_mow_not_null_t - select * except(kaint, kmintint) from src where id is not null''' + select * except(kaint, kmintint, kjson) from src where id is not null''' sql 'sync' qt_lsc4 'select * from uni_light_sc_mow_not_null_t order by id, kint' diff --git a/regression-test/suites/nereids_p0/insert_into_table/unsupport_type.groovy b/regression-test/suites/nereids_p0/insert_into_table/unsupport_type.groovy index 6edff4132dddd96..e62cfa777838f06 100644 --- a/regression-test/suites/nereids_p0/insert_into_table/unsupport_type.groovy +++ b/regression-test/suites/nereids_p0/insert_into_table/unsupport_type.groovy @@ -24,7 +24,7 @@ suite("nereids_insert_unsupport_type") { sql 'set enable_strict_consistency_dml=true' test { - sql 'insert into map_t select id, kmintint from src' + sql 'insert into json_t select id, kjson from src' exception 'unsupported for Nereids' } diff --git a/regression-test/suites/nereids_p0/subquery/test_subquery_in_project.groovy b/regression-test/suites/nereids_p0/subquery/test_subquery_in_project.groovy new file mode 100644 index 000000000000000..0521334d8ae8810 --- /dev/null +++ b/regression-test/suites/nereids_p0/subquery/test_subquery_in_project.groovy @@ -0,0 +1,120 @@ +// 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_subquery_in_project") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql """drop table if exists test_sql;""" + sql """ + CREATE TABLE `test_sql` ( + `user_id` varchar(10) NULL, + `dt` date NULL, + `city` varchar(20) NULL, + `age` int(11) NULL + ) ENGINE=OLAP + UNIQUE KEY(`user_id`) + COMMENT 'test' + DISTRIBUTED BY HASH(`user_id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "is_being_synced" = "false", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + + sql """ insert into test_sql values (1,'2020-09-09',2,3);""" + + qt_sql1 """ + select (select age from test_sql) col from test_sql order by col; + """ + + qt_sql2 """ + select (select sum(age) from test_sql) col from test_sql order by col; + """ + + qt_sql3 """ + select (select sum(age) from test_sql t2 where t2.dt = t1.dt ) col from test_sql t1 order by col; + """ + + qt_sql4 """ + select age in (select user_id from test_sql) col from test_sql order by col; + """ + + qt_sql5 """ + select age in (select user_id from test_sql t2 where t2.user_id = t1.age) col from test_sql t1 order by col; + """ + + qt_sql6 """ + select exists ( select user_id from test_sql ) col from test_sql order by col; + """ + + qt_sql7 """ + select case when age in (select user_id from test_sql) or age in (select user_id from test_sql t2 where t2.user_id = t1.age) or exists ( select user_id from test_sql ) or exists ( select t2.user_id from test_sql t2 where t2.age = t1.user_id) or age < (select sum(age) from test_sql t2 where t2.dt = t1.dt ) then 2 else 1 end col from test_sql t1 order by col; + """ + + sql """ insert into test_sql values (2,'2020-09-09',2,1);""" + + try { + sql """ + select (select age from test_sql) col from test_sql order by col; + """ + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("Expected EQ 1 to be returned by expression")) + } + + qt_sql8 """ + select (select sum(age) from test_sql) col from test_sql order by col; + """ + + qt_sql9 """ + select (select sum(age) from test_sql t2 where t2.dt = t1.dt ) col from test_sql t1 order by col; + """ + + qt_sql10 """ + select age in (select user_id from test_sql) col from test_sql order by col; + """ + + qt_sql11 """ + select age in (select user_id from test_sql t2 where t2.user_id = t1.age) col from test_sql t1 order by col; + """ + + qt_sql12 """ + select exists ( select user_id from test_sql ) col from test_sql order by col; + """ + + qt_sql13 """ + select case when age in (select user_id from test_sql) or age in (select user_id from test_sql t2 where t2.user_id = t1.age) or exists ( select user_id from test_sql ) or exists ( select t2.user_id from test_sql t2 where t2.age = t1.user_id) or age < (select sum(age) from test_sql t2 where t2.dt = t1.dt ) then 2 else 1 end col from test_sql t1 order by col; + """ + + qt_sql14 """ + select dt,case when 'med'='med' then ( + select sum(midean) from ( + select sum(score) / count(*) as midean + from ( + select age score,row_number() over (order by age desc) as desc_math, + row_number() over (order by age asc) as asc_math from test_sql + ) as order_table + where asc_math in (desc_math, desc_math + 1, desc_math - 1)) m + ) + end 'test' from test_sql group by cube(dt) order by dt; + """ + + sql """drop table if exists test_sql;""" +} diff --git a/regression-test/suites/nereids_p0/update/update_cte.groovy b/regression-test/suites/nereids_p0/update/update_cte.groovy new file mode 100644 index 000000000000000..727c079a3d83473 --- /dev/null +++ b/regression-test/suites/nereids_p0/update/update_cte.groovy @@ -0,0 +1,109 @@ +// 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('nereids_update_cte') { + def t1 = 't1_cte' + def t2 = 't2_cte' + def t3 = 't3_cte' + + sql "drop table if exists ${t1}" + sql """ + create table ${t1} ( + id int, + id1 int, + c1 bigint, + c2 string, + c3 double, + c4 date + ) unique key (id, id1) + distributed by hash(id, id1) + properties( + "replication_num"="1", + "function_column.sequence_col" = "c4" + ); + """ + + sql "drop table if exists ${t2}" + sql """ + create table ${t2} ( + id int, + c1 bigint, + c2 string, + c3 double, + c4 date + ) unique key (id) + distributed by hash(id) + properties( + "replication_num"="1" + ); + """ + + sql "drop table if exists ${t3}" + sql """ + create table ${t3} ( + id int + ) distributed by hash(id) + properties( + "replication_num"="1" + ); + """ + + sql """ + INSERT INTO ${t1} VALUES + (1, 10, 1, '1', 1.0, '2000-01-01'), + (2, 20, 2, '2', 2.0, '2000-01-02'), + (3, 30, 3, '3', 3.0, '2000-01-03'); + """ + + sql """ + + INSERT INTO ${t2} VALUES + (1, 10, '10', 10.0, '2000-01-10'), + (2, 20, '20', 20.0, '2000-01-20'), + (3, 30, '30', 30.0, '2000-01-30'), + (4, 4, '4', 4.0, '2000-01-04'), + (5, 5, '5', 5.0, '2000-01-05'); + """ + + sql """ + INSERT INTO ${t3} VALUES + (1), + (4), + (5); + """ + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_nereids_dml=true" + + sql "update ${t1} set c1 = 5 where id = 3" + + qt_sql "select * from ${t1} order by id" + + sql "update ${t1} set c1 = c1 + 1, c3 = c2 * 2 where id = 1" + + qt_sql "select * from ${t1} order by id" + + sql """ + with cte as (select * from ${t3}) + update ${t1} + set ${t1}.c1 = ${t2}.c1, ${t1}.c3 = ${t2}.c3 * 100 + from ${t2} inner join cte on ${t2}.id = cte.id + where ${t1}.id = ${t2}.id; + """ + + qt_sql "select * from ${t1} order by id" +} diff --git a/regression-test/suites/nereids_p0/with/test_with_and_two_phase_agg.groovy b/regression-test/suites/nereids_p0/with/test_with_and_two_phase_agg.groovy index c7b5865219a0aad..6b80546bc70185d 100644 --- a/regression-test/suites/nereids_p0/with/test_with_and_two_phase_agg.groovy +++ b/regression-test/suites/nereids_p0/with/test_with_and_two_phase_agg.groovy @@ -22,21 +22,21 @@ suite("test_with_and_two_phase_agg") { sql """ DROP TABLE IF EXISTS ${tableName} """ sql """ CREATE TABLE IF NOT EXISTS ${tableName}( - `key` int not null, + `key1` int not null, `key2` varchar(50) not null, `account` varchar(50) not null ) ENGINE = OLAP - UNIQUE KEY (`key`, `key2`) - DISTRIBUTED BY HASH(`key`) + UNIQUE KEY (`key1`, `key2`) + DISTRIBUTED BY HASH(`key1`) PROPERTIES("replication_num" = "1"); """ sql """ INSERT INTO ${tableName} VALUES (1, '1332050726', '1332050726'); """ qt_select """ - WITH t2 AS( SELECT sum(`key`) num, COUNT(DISTINCT `account`) unt + WITH t2 AS( SELECT sum(`key1`) num, COUNT(DISTINCT `account`) unt FROM ${tableName}) SELECT num FROM t2; """ qt_select2 """ - WITH t2 AS( SELECT `key2`, sum(`key`) num, COUNT(DISTINCT `account`) unt + WITH t2 AS( SELECT `key2`, sum(`key1`) num, COUNT(DISTINCT `account`) unt FROM ${tableName} GROUP BY `key2`) SELECT num FROM t2; """ } diff --git a/regression-test/suites/nereids_syntax_p0/analyze_agg.groovy b/regression-test/suites/nereids_syntax_p0/analyze_agg.groovy index 2f0b0d01c639c8c..55bb2deab2384bf 100644 --- a/regression-test/suites/nereids_syntax_p0/analyze_agg.groovy +++ b/regression-test/suites/nereids_syntax_p0/analyze_agg.groovy @@ -70,7 +70,7 @@ suite("analyze_agg") { """ test { - sql "select count(distinct t2.id), max(distinct t2.c) from t2" - exception "max(DISTINCT c#2) can't support multi distinct." + sql "select count(distinct t2.b), variance(distinct t2.c) from t2" + exception "variance(DISTINCT c#2) can't support multi distinct." } } \ No newline at end of file diff --git a/regression-test/suites/nereids_syntax_p0/collate.groovy b/regression-test/suites/nereids_syntax_p0/collate.groovy new file mode 100644 index 000000000000000..c05cb4838df5b43 --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/collate.groovy @@ -0,0 +1,25 @@ +// 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("collate_grammar_test") { + sql "set enable_nereids_planner=true;" + sql "set enable_fallback_to_original_planner=false;" + sql """ + select table_name from information_schema.tables where table_schema collate utf8_general_ci = 'information_schema' and table_name collate utf8_general_ci = 'parameters'; + """ + +} \ No newline at end of file diff --git a/regression-test/suites/nereids_syntax_p0/cte.groovy b/regression-test/suites/nereids_syntax_p0/cte.groovy index 15a7afc9556edbc..457ecc2801484d1 100644 --- a/regression-test/suites/nereids_syntax_p0/cte.groovy +++ b/regression-test/suites/nereids_syntax_p0/cte.groovy @@ -306,5 +306,6 @@ suite("cte") { notContains "MultiCastDataSinks" } + sql "WITH cte_0 AS ( SELECT 1 AS a ) SELECT * from cte_0 t1 LIMIT 10 UNION SELECT * from cte_0 t1 LIMIT 10" } diff --git a/regression-test/suites/nereids_syntax_p0/test_complex_type.groovy b/regression-test/suites/nereids_syntax_p0/test_complex_type.groovy new file mode 100644 index 000000000000000..145e3d2eb3b1e60 --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/test_complex_type.groovy @@ -0,0 +1,63 @@ +// 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_complex_type") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + + // array literal + sql "select [1, 2, 3, 4]" + + // map literal + sql "select {1:2, 3:4}" + + // struct literal + sql "select {1, 2, 3, 4}" + + // struct constructor + sql "select struct(1, 2, 3, 4)" + + // named struct constructor + sql "select named_struct('x', 1, 'y', 2)" + + test { + sql "select named_struct('x', 1, 'x', 2)" + exception "The name of the struct field cannot be repeated" + } + + // struct element with int + sql "select struct_element(struct('1', '2', '3', '4'), 1);" + + test { + sql "select struct_element(struct('1', '2', '3', '4'), 5);" + exception "the specified field index out of bound" + } + + test { + sql "select struct_element(struct('1', '2', '3', '4'), -1);" + exception "the specified field index out of bound" + } + + // struct element with string + sql "select struct_element(named_struct('1', '2', '3', '4'), '1');" + + test { + sql "select struct_element(named_struct('1', '2', '3', '4'), '5')" + exception "the specified field name 5 was not found" + } +} + 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 53c0ff9a1257789..4b3cd3bdca18213 100644 --- a/regression-test/suites/nereids_syntax_p0/test_simplify_comparison.groovy +++ b/regression-test/suites/nereids_syntax_p0/test_simplify_comparison.groovy @@ -72,4 +72,252 @@ suite("test_simplify_comparison") { } sql "select cast('1234' as decimalv3(18,4)) > 2000;" + + sql 'drop table if exists simple_test_table_t;' + sql """CREATE TABLE IF NOT EXISTS `simple_test_table_t` ( + a tinyint, + b smallint, + c int, + d bigint, + e largeint + ) ENGINE=OLAP + UNIQUE KEY (`a`) + DISTRIBUTED BY HASH(`a`) BUCKETS 120 + PROPERTIES ( + "replication_num" = "1", + "in_memory" = "false", + "compression" = "LZ4" + );""" + + explain { + sql "verbose select * from simple_test_table_t where a = cast(1.0 as double) and b = cast(1.0 as double) and c = cast(1.0 as double) and d = cast(1.0 as double);" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e = cast(1.0 as double);" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a > cast(1.0 as double) and b > cast(1.0 as double) and c > cast(1.0 as double) and d > cast(1.0 as double);" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e > cast(1.0 as double);" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a < cast(1.0 as double) and b < cast(1.0 as double) and c < cast(1.0 as double) and d < cast(1.0 as double);" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e < cast(1.0 as double);" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a >= cast(1.0 as double) and b >= cast(1.0 as double) and c >= cast(1.0 as double) and d >= cast(1.0 as double);" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e >= cast(1.0 as double);" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a <= cast(1.0 as double) and b <= cast(1.0 as double) and c <= cast(1.0 as double) and d <= cast(1.0 as double);" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e <= cast(1.0 as double);" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a = cast(1.1 as double) and b = cast(1.1 as double) and c = cast(1.1 as double) and d = cast(1.1 as double);" + contains "a[#0] IS NULL" + contains "b[#1] IS NULL" + contains "c[#2] IS NULL" + contains "d[#3] IS NULL" + contains "AND NULL" + } + + explain { + sql "verbose select * from simple_test_table_t where e = cast(1.1 as double);" + contains "CAST(e[#4] AS DOUBLE) = 1.1" + } + + explain { + sql "verbose select * from simple_test_table_t where a > cast(1.1 as double) and b > cast(1.1 as double) and c > cast(1.1 as double) and d > cast(1.1 as double);" + contains "a[#0] > 1" + contains "b[#1] > 1" + contains "c[#2] > 1" + contains "d[#3] > 1" + } + + explain { + sql "verbose select * from simple_test_table_t where e > cast(1.1 as double);" + contains "CAST(e[#4] AS DOUBLE) > 1.1" + } + + explain { + sql "verbose select * from simple_test_table_t where a < cast(1.1 as double) and b < cast(1.1 as double) and c < cast(1.1 as double) and d < cast(1.1 as double);" + contains "a[#0] < 2" + contains "b[#1] < 2" + contains "c[#2] < 2" + contains "d[#3] < 2" + } + + explain { + sql "verbose select * from simple_test_table_t where e < cast(1.1 as double);" + contains "CAST(e[#4] AS DOUBLE) < 1.1" + } + + explain { + sql "verbose select * from simple_test_table_t where a >= cast(1.1 as double) and b >= cast(1.1 as double) and c >= cast(1.1 as double) and d >= cast(1.1 as double);" + contains "a[#0] >= 2" + contains "b[#1] >= 2" + contains "c[#2] >= 2" + contains "d[#3] >= 2" + } + + explain { + sql "verbose select * from simple_test_table_t where e >= cast(1.1 as double);" + contains "CAST(e[#4] AS DOUBLE) >= 1.1" + } + + explain { + sql "verbose select * from simple_test_table_t where a <= cast(1.1 as double) and b <= cast(1.1 as double) and c <= cast(1.1 as double) and d <= cast(1.1 as double);" + contains "a[#0] <= 1" + contains "b[#1] <= 1" + contains "c[#2] <= 1" + contains "d[#3] <= 1" + } + + explain { + sql "verbose select * from simple_test_table_t where e <= cast(1.1 as double);" + contains "CAST(e[#4] AS DOUBLE) <= 1.1" + } + + explain { + sql "verbose select * from simple_test_table_t where a = 1.0 and b = 1.0 and c = 1.0 and d = 1.0;" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e = 1.0;" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a > 1.0 and b > 1.0 and c > 1.0 and d > 1.0;" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e > 1.0;" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a < 1.0 and b < 1.0 and c < 1.0 and d < 1.0;" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e < 1.0;" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a >= 1.0 and b >= 1.0 and c >= 1.0 and d >= 1.0;" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e >= 1.0;" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a <= 1.0 and b <= 1.0 and c <= 1.0 and d <= 1.0;" + notContains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where e <= 1.0;" + contains "CAST" + } + + explain { + sql "verbose select * from simple_test_table_t where a = 1.1 and b = 1.1 and c = 1.1 and d = 1.1;" + contains "a[#0] IS NULL" + contains "b[#1] IS NULL" + contains "c[#2] IS NULL" + contains "d[#3] IS NULL" + contains "AND NULL" + } + + explain { + sql "verbose select * from simple_test_table_t where e = 1.1;" + contains "CAST(e[#4] AS DOUBLE) = 1.1" + } + + explain { + sql "verbose select * from simple_test_table_t where a > 1.1 and b > 1.1 and c > 1.1 and d > 1.1;" + contains "a[#0] > 1" + contains "b[#1] > 1" + contains "c[#2] > 1" + contains "d[#3] > 1" + } + + explain { + sql "verbose select * from simple_test_table_t where e > 1.1;" + contains "CAST(e[#4] AS DOUBLE) > 1.1" + } + + explain { + sql "verbose select * from simple_test_table_t where a < 1.1 and b < 1.1 and c < 1.1 and d < 1.1;" + contains "a[#0] < 2" + contains "b[#1] < 2" + contains "c[#2] < 2" + contains "d[#3] < 2" + } + + explain { + sql "verbose select * from simple_test_table_t where e < 1.1;" + contains "CAST(e[#4] AS DOUBLE) < 1.1" + } + + explain { + sql "verbose select * from simple_test_table_t where a >= 1.1 and b >= 1.1 and c >= 1.1 and d >= 1.1;" + contains "a[#0] >= 2" + contains "b[#1] >= 2" + contains "c[#2] >= 2" + contains "d[#3] >= 2" + } + + explain { + sql "verbose select * from simple_test_table_t where e >= 1.1;" + contains "CAST(e[#4] AS DOUBLE) >= 1.1" + } + + explain { + sql "verbose select * from simple_test_table_t where a <= 1.1 and b <= 1.1 and c <= 1.1 and d <= 1.1;" + contains "a[#0] <= 1" + contains "b[#1] <= 1" + contains "c[#2] <= 1" + contains "d[#3] <= 1" + } + + explain { + sql "verbose select * from simple_test_table_t where e <= 1.1;" + contains "CAST(e[#4] AS DOUBLE) <= 1.1" + } } \ No newline at end of file diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/gen_rf.py b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/gen_rf.py new file mode 100644 index 000000000000000..bb96672a0ed169c --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/gen_rf.py @@ -0,0 +1,29 @@ +# // 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. +if __name__ == '__main__': + with open('rf.tmpl', 'r') as f: + tmpl = f.read() + for i in range(1,100): + with open('../../../../tools/tpcds-tools/queries/sf100/query'+str(i)+'.sql', 'r') as fi: + casei = tmpl.replace('{--}', str(i)) + casei = casei.replace('{query}', fi.read()) + # with open('../rf/ds_rf'+str(i)+'.groovy', 'w') as out: + # out.write(casei) + with open('rf/rf.'+str(i), 'r') as rf_file: + casei = casei.replace('{rfs}', rf_file.read()) + with open('../rf/ds_rf'+str(i)+'.groovy', 'w') as out: + out.write(casei) \ No newline at end of file diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/gen_cases.py b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/gen_shape.py similarity index 86% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/gen_cases.py rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/gen_shape.py index d570d375f465a6d..d56bd6461d52608 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/gen_cases.py +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/gen_shape.py @@ -15,10 +15,10 @@ # // specific language governing permissions and limitations # // under the License. if __name__ == '__main__': - with open('case.tmpl', 'r') as f: + with open('shape.tmpl', 'r') as f: tmpl = f.read() for i in range(1,100): - with open('query'+str(i)+'.sql', 'r') as fi: + with open('../../../../tools/tpcds-tools/queries/sf100/query'+str(i)+'.sql', 'r') as fi: casei = tmpl.replace('{--}', str(i)) casei = casei.replace('{query}', fi.read()) diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query1.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query1.sql deleted file mode 100644 index aa95b7c934dab4e..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query1.sql +++ /dev/null @@ -1,23 +0,0 @@ -explain shape plan -with customer_total_return as -(select sr_customer_sk as ctr_customer_sk -,sr_store_sk as ctr_store_sk -,sum(SR_FEE) as ctr_total_return -from store_returns -,date_dim -where sr_returned_date_sk = d_date_sk -and d_year =2000 -group by sr_customer_sk -,sr_store_sk) - select c_customer_id -from customer_total_return ctr1 -,store -,customer -where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 -from customer_total_return ctr2 -where ctr1.ctr_store_sk = ctr2.ctr_store_sk) -and s_store_sk = ctr1.ctr_store_sk -and s_state = 'SD' -and ctr1.ctr_customer_sk = c_customer_sk -order by c_customer_id -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query10.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query10.sql deleted file mode 100644 index a90f1269fbaa14d..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query10.sql +++ /dev/null @@ -1,61 +0,0 @@ -explain shape plan - - - - -select - cd_gender, - cd_marital_status, - cd_education_status, - count(*) cnt1, - cd_purchase_estimate, - count(*) cnt2, - cd_credit_rating, - count(*) cnt3, - cd_dep_count, - count(*) cnt4, - cd_dep_employed_count, - count(*) cnt5, - cd_dep_college_count, - count(*) cnt6 - from - customer c,customer_address ca,customer_demographics - where - c.c_current_addr_sk = ca.ca_address_sk and - ca_county in ('Storey County','Marquette County','Warren County','Cochran County','Kandiyohi County') and - cd_demo_sk = c.c_current_cdemo_sk and - exists (select * - from store_sales,date_dim - where c.c_customer_sk = ss_customer_sk and - ss_sold_date_sk = d_date_sk and - d_year = 2001 and - d_moy between 1 and 1+3) and - (exists (select * - from web_sales,date_dim - where c.c_customer_sk = ws_bill_customer_sk and - ws_sold_date_sk = d_date_sk and - d_year = 2001 and - d_moy between 1 ANd 1+3) or - exists (select * - from catalog_sales,date_dim - where c.c_customer_sk = cs_ship_customer_sk and - cs_sold_date_sk = d_date_sk and - d_year = 2001 and - d_moy between 1 and 1+3)) - group by cd_gender, - cd_marital_status, - cd_education_status, - cd_purchase_estimate, - cd_credit_rating, - cd_dep_count, - cd_dep_employed_count, - cd_dep_college_count - order by cd_gender, - cd_marital_status, - cd_education_status, - cd_purchase_estimate, - cd_credit_rating, - cd_dep_count, - cd_dep_employed_count, - cd_dep_college_count -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query12.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query12.sql deleted file mode 100644 index 3259e805b52a3f9..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query12.sql +++ /dev/null @@ -1,36 +0,0 @@ -explain shape plan - - - - -select i_item_id - ,i_item_desc - ,i_category - ,i_class - ,i_current_price - ,sum(ws_ext_sales_price) as itemrevenue - ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over - (partition by i_class) as revenueratio -from - web_sales - ,item - ,date_dim -where - ws_item_sk = i_item_sk - and i_category in ('Books', 'Sports', 'Men') - and ws_sold_date_sk = d_date_sk - and d_date between cast('1998-04-06' as date) - and (cast('1998-04-06' as date) + interval 30 day) -group by - i_item_id - ,i_item_desc - ,i_category - ,i_class - ,i_current_price -order by - i_category - ,i_class - ,i_item_id - ,i_item_desc - ,revenueratio -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query13.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query13.sql deleted file mode 100644 index 0241faf922d16ac..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query13.sql +++ /dev/null @@ -1,52 +0,0 @@ -explain shape plan - - -select avg(ss_quantity) - ,avg(ss_ext_sales_price) - ,avg(ss_ext_wholesale_cost) - ,sum(ss_ext_wholesale_cost) - from store_sales - ,store - ,customer_demographics - ,household_demographics - ,customer_address - ,date_dim - where s_store_sk = ss_store_sk - and ss_sold_date_sk = d_date_sk and d_year = 2001 - and((ss_hdemo_sk=hd_demo_sk - and cd_demo_sk = ss_cdemo_sk - and cd_marital_status = 'D' - and cd_education_status = 'Unknown' - and ss_sales_price between 100.00 and 150.00 - and hd_dep_count = 3 - )or - (ss_hdemo_sk=hd_demo_sk - and cd_demo_sk = ss_cdemo_sk - and cd_marital_status = 'S' - and cd_education_status = 'College' - and ss_sales_price between 50.00 and 100.00 - and hd_dep_count = 1 - ) or - (ss_hdemo_sk=hd_demo_sk - and cd_demo_sk = ss_cdemo_sk - and cd_marital_status = 'M' - and cd_education_status = '4 yr Degree' - and ss_sales_price between 150.00 and 200.00 - and hd_dep_count = 1 - )) - and((ss_addr_sk = ca_address_sk - and ca_country = 'United States' - and ca_state in ('SD', 'KS', 'MI') - and ss_net_profit between 100 and 200 - ) or - (ss_addr_sk = ca_address_sk - and ca_country = 'United States' - and ca_state in ('MO', 'ND', 'CO') - and ss_net_profit between 150 and 300 - ) or - (ss_addr_sk = ca_address_sk - and ca_country = 'United States' - and ca_state in ('NH', 'OH', 'TX') - and ss_net_profit between 50 and 250 - )) -; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query15.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query15.sql deleted file mode 100644 index f8bb58f52b030c2..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query15.sql +++ /dev/null @@ -1,21 +0,0 @@ -explain shape plan - - - -select ca_zip - ,sum(cs_sales_price) - from catalog_sales - ,customer - ,customer_address - ,date_dim - where cs_bill_customer_sk = c_customer_sk - and c_current_addr_sk = ca_address_sk - and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', - '85392', '85460', '80348', '81792') - or ca_state in ('CA','WA','GA') - or cs_sales_price > 500) - and cs_sold_date_sk = d_date_sk - and d_qoy = 1 and d_year = 2001 - group by ca_zip - order by ca_zip - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query16.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query16.sql deleted file mode 100644 index 95d8b5a847d6893..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query16.sql +++ /dev/null @@ -1,33 +0,0 @@ -explain shape plan - - - - -select - count(distinct cs_order_number) as "order count" - ,sum(cs_ext_ship_cost) as "total shipping cost" - ,sum(cs_net_profit) as "total net profit" -from - catalog_sales cs1 - ,date_dim - ,customer_address - ,call_center -where - d_date between '2002-4-01' and - (cast('2002-4-01' as date) + interval 60 day) -and cs1.cs_ship_date_sk = d_date_sk -and cs1.cs_ship_addr_sk = ca_address_sk -and ca_state = 'WV' -and cs1.cs_call_center_sk = cc_call_center_sk -and cc_county in ('Ziebach County','Luce County','Richland County','Daviess County', - 'Barrow County' -) -and exists (select * - from catalog_sales cs2 - where cs1.cs_order_number = cs2.cs_order_number - and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) -and not exists(select * - from catalog_returns cr1 - where cs1.cs_order_number = cr1.cr_order_number) -order by count(distinct cs_order_number) -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query17.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query17.sql deleted file mode 100644 index 52feda4937e8d94..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query17.sql +++ /dev/null @@ -1,47 +0,0 @@ -explain shape plan - - - - -select i_item_id - ,i_item_desc - ,s_state - ,count(ss_quantity) as store_sales_quantitycount - ,avg(ss_quantity) as store_sales_quantityave - ,stddev_samp(ss_quantity) as store_sales_quantitystdev - ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov - ,count(sr_return_quantity) as store_returns_quantitycount - ,avg(sr_return_quantity) as store_returns_quantityave - ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev - ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov - ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave - ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev - ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov - from store_sales - ,store_returns - ,catalog_sales - ,date_dim d1 - ,date_dim d2 - ,date_dim d3 - ,store - ,item - where d1.d_quarter_name = '2001Q1' - and d1.d_date_sk = ss_sold_date_sk - and i_item_sk = ss_item_sk - and s_store_sk = ss_store_sk - and ss_customer_sk = sr_customer_sk - and ss_item_sk = sr_item_sk - and ss_ticket_number = sr_ticket_number - and sr_returned_date_sk = d2.d_date_sk - and d2.d_quarter_name in ('2001Q1','2001Q2','2001Q3') - and sr_customer_sk = cs_bill_customer_sk - and sr_item_sk = cs_item_sk - and cs_sold_date_sk = d3.d_date_sk - and d3.d_quarter_name in ('2001Q1','2001Q2','2001Q3') - group by i_item_id - ,i_item_desc - ,s_state - order by i_item_id - ,i_item_desc - ,s_state -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query18.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query18.sql deleted file mode 100644 index 86809ad5b6870d7..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query18.sql +++ /dev/null @@ -1,36 +0,0 @@ -explain shape plan - - - - -select i_item_id, - ca_country, - ca_state, - ca_county, - avg( cast(cs_quantity as decimal(12,2))) agg1, - avg( cast(cs_list_price as decimal(12,2))) agg2, - avg( cast(cs_coupon_amt as decimal(12,2))) agg3, - avg( cast(cs_sales_price as decimal(12,2))) agg4, - avg( cast(cs_net_profit as decimal(12,2))) agg5, - avg( cast(c_birth_year as decimal(12,2))) agg6, - avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 - from catalog_sales, customer_demographics cd1, - customer_demographics cd2, customer, customer_address, date_dim, item - where cs_sold_date_sk = d_date_sk and - cs_item_sk = i_item_sk and - cs_bill_cdemo_sk = cd1.cd_demo_sk and - cs_bill_customer_sk = c_customer_sk and - cd1.cd_gender = 'F' and - cd1.cd_education_status = 'Advanced Degree' and - c_current_cdemo_sk = cd2.cd_demo_sk and - c_current_addr_sk = ca_address_sk and - c_birth_month in (10,7,8,4,1,2) and - d_year = 1998 and - ca_state in ('WA','GA','NC' - ,'ME','WY','OK','IN') - group by rollup (i_item_id, ca_country, ca_state, ca_county) - order by ca_country, - ca_state, - ca_county, - i_item_id - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query19.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query19.sql deleted file mode 100644 index df66b1102cfff28..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query19.sql +++ /dev/null @@ -1,27 +0,0 @@ -explain shape plan - - - - -select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, - sum(ss_ext_sales_price) ext_price - from date_dim, store_sales, item,customer,customer_address,store - where d_date_sk = ss_sold_date_sk - and ss_item_sk = i_item_sk - and i_manager_id=2 - and d_moy=12 - and d_year=1999 - and ss_customer_sk = c_customer_sk - and c_current_addr_sk = ca_address_sk - and substr(ca_zip,1,5) <> substr(s_zip,1,5) - and ss_store_sk = s_store_sk - group by i_brand - ,i_brand_id - ,i_manufact_id - ,i_manufact - order by ext_price desc - ,i_brand - ,i_brand_id - ,i_manufact_id - ,i_manufact -limit 100 ; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query2.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query2.sql deleted file mode 100644 index ef94f1394f13719..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query2.sql +++ /dev/null @@ -1,62 +0,0 @@ -explain shape plan - - - - -with wscs as - (select sold_date_sk - ,sales_price - from (select ws_sold_date_sk sold_date_sk - ,ws_ext_sales_price sales_price - from web_sales - union all - select cs_sold_date_sk sold_date_sk - ,cs_ext_sales_price sales_price - from catalog_sales) t), - wswscs as - (select d_week_seq, - sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, - sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, - sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, - sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, - sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, - sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, - sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales - from wscs - ,date_dim - where d_date_sk = sold_date_sk - group by d_week_seq) - select d_week_seq1 - ,round(sun_sales1/sun_sales2,2) - ,round(mon_sales1/mon_sales2,2) - ,round(tue_sales1/tue_sales2,2) - ,round(wed_sales1/wed_sales2,2) - ,round(thu_sales1/thu_sales2,2) - ,round(fri_sales1/fri_sales2,2) - ,round(sat_sales1/sat_sales2,2) - from - (select wswscs.d_week_seq d_week_seq1 - ,sun_sales sun_sales1 - ,mon_sales mon_sales1 - ,tue_sales tue_sales1 - ,wed_sales wed_sales1 - ,thu_sales thu_sales1 - ,fri_sales fri_sales1 - ,sat_sales sat_sales1 - from wswscs,date_dim - where date_dim.d_week_seq = wswscs.d_week_seq and - d_year = 1998) y, - (select wswscs.d_week_seq d_week_seq2 - ,sun_sales sun_sales2 - ,mon_sales mon_sales2 - ,tue_sales tue_sales2 - ,wed_sales wed_sales2 - ,thu_sales thu_sales2 - ,fri_sales fri_sales2 - ,sat_sales sat_sales2 - from wswscs - ,date_dim - where date_dim.d_week_seq = wswscs.d_week_seq and - d_year = 1998+1) z - where d_week_seq1=d_week_seq2-53 - order by d_week_seq1; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query20.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query20.sql deleted file mode 100644 index 314d9043ec1d5c5..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query20.sql +++ /dev/null @@ -1,32 +0,0 @@ -explain shape plan - - - - -select i_item_id - ,i_item_desc - ,i_category - ,i_class - ,i_current_price - ,sum(cs_ext_sales_price) as itemrevenue - ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over - (partition by i_class) as revenueratio - from catalog_sales - ,item - ,date_dim - where cs_item_sk = i_item_sk - and i_category in ('Shoes', 'Books', 'Women') - and cs_sold_date_sk = d_date_sk - and d_date between cast('2002-01-26' as date) - and (cast('2002-01-26' as date) + interval 30 day) - group by i_item_id - ,i_item_desc - ,i_category - ,i_class - ,i_current_price - order by i_category - ,i_class - ,i_item_id - ,i_item_desc - ,revenueratio -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query21.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query21.sql deleted file mode 100644 index 17be7bfdac6a047..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query21.sql +++ /dev/null @@ -1,32 +0,0 @@ -explain shape plan - - - - -select * - from(select w_warehouse_name - ,i_item_id - ,sum(case when (cast(d_date as date) < cast ('2002-02-27' as date)) - then inv_quantity_on_hand - else 0 end) as inv_before - ,sum(case when (cast(d_date as date) >= cast ('2002-02-27' as date)) - then inv_quantity_on_hand - else 0 end) as inv_after - from inventory - ,warehouse - ,item - ,date_dim - where i_current_price between 0.99 and 1.49 - and i_item_sk = inv_item_sk - and inv_warehouse_sk = w_warehouse_sk - and inv_date_sk = d_date_sk - and d_date between (cast ('2002-02-27' as date) - interval 30 day) - and (cast ('2002-02-27' as date) + interval 30 day) - group by w_warehouse_name, i_item_id) x - where (case when inv_before > 0 - then inv_after / inv_before - else null - end) between 2.0/3.0 and 3.0/2.0 - order by w_warehouse_name - ,i_item_id - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query22.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query22.sql deleted file mode 100644 index 21aa6e826cd16c2..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query22.sql +++ /dev/null @@ -1,22 +0,0 @@ -explain shape plan - - - - -select i_product_name - ,i_brand - ,i_class - ,i_category - ,avg(inv_quantity_on_hand) qoh - from inventory - ,date_dim - ,item - where inv_date_sk=d_date_sk - and inv_item_sk=i_item_sk - and d_month_seq between 1188 and 1188 + 11 - group by rollup(i_product_name - ,i_brand - ,i_class - ,i_category) -order by qoh, i_product_name, i_brand, i_class, i_category -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query23.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query23.sql deleted file mode 100644 index 24166d2de90966e..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query23.sql +++ /dev/null @@ -1,53 +0,0 @@ -explain shape plan - - - -with frequent_ss_items as - (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt - from store_sales - ,date_dim - ,item - where ss_sold_date_sk = d_date_sk - and ss_item_sk = i_item_sk - and d_year in (2000,2000+1,2000+2,2000+3) - group by substr(i_item_desc,1,30),i_item_sk,d_date - having count(*) >4), - max_store_sales as - (select max(csales) tpcds_cmax - from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales - from store_sales - ,customer - ,date_dim - where ss_customer_sk = c_customer_sk - and ss_sold_date_sk = d_date_sk - and d_year in (2000,2000+1,2000+2,2000+3) - group by c_customer_sk) t), - best_ss_customer as - (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales - from store_sales - ,customer - where ss_customer_sk = c_customer_sk - group by c_customer_sk - having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select - * -from - max_store_sales)) - select sum(sales) - from (select cs_quantity*cs_list_price sales - from catalog_sales - ,date_dim - where d_year = 2000 - and d_moy = 5 - and cs_sold_date_sk = d_date_sk - and cs_item_sk in (select item_sk from frequent_ss_items) - and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) - union all - select ws_quantity*ws_list_price sales - from web_sales - ,date_dim - where d_year = 2000 - and d_moy = 5 - and ws_sold_date_sk = d_date_sk - and ws_item_sk in (select item_sk from frequent_ss_items) - and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) t2 - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query24.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query24.sql deleted file mode 100644 index a520f4b745bf826..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query24.sql +++ /dev/null @@ -1,56 +0,0 @@ - explain shape plan - - - - with ssales as - (select c_last_name - ,c_first_name - ,s_store_name - ,ca_state - ,s_state - ,i_color - ,i_current_price - ,i_manager_id - ,i_units - ,i_size - ,sum(ss_net_profit) netpaid - from store_sales - ,store_returns - ,store - ,item - ,customer - ,customer_address - where ss_ticket_number = sr_ticket_number - and ss_item_sk = sr_item_sk - and ss_customer_sk = c_customer_sk - and ss_item_sk = i_item_sk - and ss_store_sk = s_store_sk - and c_current_addr_sk = ca_address_sk - and c_birth_country <> upper(ca_country) - and s_zip = ca_zip - and s_market_id=8 - group by c_last_name - ,c_first_name - ,s_store_name - ,ca_state - ,s_state - ,i_color - ,i_current_price - ,i_manager_id - ,i_units - ,i_size) - select c_last_name - ,c_first_name - ,s_store_name - ,sum(netpaid) paid - from ssales - where i_color = 'beige' - group by c_last_name - ,c_first_name - ,s_store_name - having sum(netpaid) > (select 0.05*avg(netpaid) - from ssales) - order by c_last_name - ,c_first_name - ,s_store_name - ; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query25.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query25.sql deleted file mode 100644 index ffd0a28111a22d4..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query25.sql +++ /dev/null @@ -1,49 +0,0 @@ -explain shape plan - - - -select - i_item_id - ,i_item_desc - ,s_store_id - ,s_store_name - ,sum(ss_net_profit) as store_sales_profit - ,sum(sr_net_loss) as store_returns_loss - ,sum(cs_net_profit) as catalog_sales_profit - from - store_sales - ,store_returns - ,catalog_sales - ,date_dim d1 - ,date_dim d2 - ,date_dim d3 - ,store - ,item - where - d1.d_moy = 4 - and d1.d_year = 2000 - and d1.d_date_sk = ss_sold_date_sk - and i_item_sk = ss_item_sk - and s_store_sk = ss_store_sk - and ss_customer_sk = sr_customer_sk - and ss_item_sk = sr_item_sk - and ss_ticket_number = sr_ticket_number - and sr_returned_date_sk = d2.d_date_sk - and d2.d_moy between 4 and 10 - and d2.d_year = 2000 - and sr_customer_sk = cs_bill_customer_sk - and sr_item_sk = cs_item_sk - and cs_sold_date_sk = d3.d_date_sk - and d3.d_moy between 4 and 10 - and d3.d_year = 2000 - group by - i_item_id - ,i_item_desc - ,s_store_id - ,s_store_name - order by - i_item_id - ,i_item_desc - ,s_store_id - ,s_store_name - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query26.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query26.sql deleted file mode 100644 index 8d3ceb5d34a1ed4..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query26.sql +++ /dev/null @@ -1,23 +0,0 @@ -explain shape plan - - - - -select i_item_id, - avg(cs_quantity) agg1, - avg(cs_list_price) agg2, - avg(cs_coupon_amt) agg3, - avg(cs_sales_price) agg4 - from catalog_sales, customer_demographics, date_dim, item, promotion - where cs_sold_date_sk = d_date_sk and - cs_item_sk = i_item_sk and - cs_bill_cdemo_sk = cd_demo_sk and - cs_promo_sk = p_promo_sk and - cd_gender = 'M' and - cd_marital_status = 'S' and - cd_education_status = 'Unknown' and - (p_channel_email = 'N' or p_channel_event = 'N') and - d_year = 2001 - group by i_item_id - order by i_item_id - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query27.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query27.sql deleted file mode 100644 index 3bca9c18c435de7..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query27.sql +++ /dev/null @@ -1,24 +0,0 @@ -explain shape plan - - - -select i_item_id, - s_state, grouping(s_state) g_state, - avg(ss_quantity) agg1, - avg(ss_list_price) agg2, - avg(ss_coupon_amt) agg3, - avg(ss_sales_price) agg4 - from store_sales, customer_demographics, date_dim, store, item - where ss_sold_date_sk = d_date_sk and - ss_item_sk = i_item_sk and - ss_store_sk = s_store_sk and - ss_cdemo_sk = cd_demo_sk and - cd_gender = 'F' and - cd_marital_status = 'D' and - cd_education_status = 'Secondary' and - d_year = 1999 and - s_state in ('MO','AL', 'MI', 'TN', 'LA', 'SC') - group by rollup (i_item_id, s_state) - order by i_item_id - ,s_state - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query29.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query29.sql deleted file mode 100644 index 7fc8f8826e7e656..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query29.sql +++ /dev/null @@ -1,49 +0,0 @@ -explain shape plan - - - - -select - i_item_id - ,i_item_desc - ,s_store_id - ,s_store_name - ,avg(ss_quantity) as store_sales_quantity - ,avg(sr_return_quantity) as store_returns_quantity - ,avg(cs_quantity) as catalog_sales_quantity - from - store_sales - ,store_returns - ,catalog_sales - ,date_dim d1 - ,date_dim d2 - ,date_dim d3 - ,store - ,item - where - d1.d_moy = 4 - and d1.d_year = 1999 - and d1.d_date_sk = ss_sold_date_sk - and i_item_sk = ss_item_sk - and s_store_sk = ss_store_sk - and ss_customer_sk = sr_customer_sk - and ss_item_sk = sr_item_sk - and ss_ticket_number = sr_ticket_number - and sr_returned_date_sk = d2.d_date_sk - and d2.d_moy between 4 and 4 + 3 - and d2.d_year = 1999 - and sr_customer_sk = cs_bill_customer_sk - and sr_item_sk = cs_item_sk - and cs_sold_date_sk = d3.d_date_sk - and d3.d_year in (1999,1999+1,1999+2) - group by - i_item_id - ,i_item_desc - ,s_store_id - ,s_store_name - order by - i_item_id - ,i_item_desc - ,s_store_id - ,s_store_name - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query3.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query3.sql deleted file mode 100644 index e08d8b00c2a1465..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query3.sql +++ /dev/null @@ -1,23 +0,0 @@ -explain shape plan - - - - -select dt.d_year - ,item.i_brand_id brand_id - ,item.i_brand brand - ,sum(ss_sales_price) sum_agg - from date_dim dt - ,store_sales - ,item - where dt.d_date_sk = store_sales.ss_sold_date_sk - and store_sales.ss_item_sk = item.i_item_sk - and item.i_manufact_id = 816 - and dt.d_moy=11 - group by dt.d_year - ,item.i_brand - ,item.i_brand_id - order by dt.d_year - ,sum_agg desc - ,brand_id - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query30.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query30.sql deleted file mode 100644 index 39b445e830d8a7f..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query30.sql +++ /dev/null @@ -1,33 +0,0 @@ -explain shape plan - - - - -with customer_total_return as - (select wr_returning_customer_sk as ctr_customer_sk - ,ca_state as ctr_state, - sum(wr_return_amt) as ctr_total_return - from web_returns - ,date_dim - ,customer_address - where wr_returned_date_sk = d_date_sk - and d_year =2002 - and wr_returning_addr_sk = ca_address_sk - group by wr_returning_customer_sk - ,ca_state) - select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag - ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address - ,c_last_review_date_sk,ctr_total_return - from customer_total_return ctr1 - ,customer_address - ,customer - where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 - from customer_total_return ctr2 - where ctr1.ctr_state = ctr2.ctr_state) - and ca_address_sk = c_current_addr_sk - and ca_state = 'IN' - and ctr1.ctr_customer_sk = c_customer_sk - order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag - ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address - ,c_last_review_date_sk,ctr_total_return -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query31.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query31.sql deleted file mode 100644 index 31d4a6d8739d50d..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query31.sql +++ /dev/null @@ -1,54 +0,0 @@ -explain shape plan - - - - -with ss as - (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales - from store_sales,date_dim,customer_address - where ss_sold_date_sk = d_date_sk - and ss_addr_sk=ca_address_sk - group by ca_county,d_qoy, d_year), - ws as - (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales - from web_sales,date_dim,customer_address - where ws_sold_date_sk = d_date_sk - and ws_bill_addr_sk=ca_address_sk - group by ca_county,d_qoy, d_year) - select - ss1.ca_county - ,ss1.d_year - ,ws2.web_sales/ws1.web_sales web_q1_q2_increase - ,ss2.store_sales/ss1.store_sales store_q1_q2_increase - ,ws3.web_sales/ws2.web_sales web_q2_q3_increase - ,ss3.store_sales/ss2.store_sales store_q2_q3_increase - from - ss ss1 - ,ss ss2 - ,ss ss3 - ,ws ws1 - ,ws ws2 - ,ws ws3 - where - ss1.d_qoy = 1 - and ss1.d_year = 2000 - and ss1.ca_county = ss2.ca_county - and ss2.d_qoy = 2 - and ss2.d_year = 2000 - and ss2.ca_county = ss3.ca_county - and ss3.d_qoy = 3 - and ss3.d_year = 2000 - and ss1.ca_county = ws1.ca_county - and ws1.d_qoy = 1 - and ws1.d_year = 2000 - and ws1.ca_county = ws2.ca_county - and ws2.d_qoy = 2 - and ws2.d_year = 2000 - and ws1.ca_county = ws3.ca_county - and ws3.d_qoy = 3 - and ws3.d_year =2000 - and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end - > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end - and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end - > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end - order by web_q1_q2_increase; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query32.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query32.sql deleted file mode 100644 index 7ed8ad140ef40dc..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query32.sql +++ /dev/null @@ -1,30 +0,0 @@ -explain shape plan - - - - -select sum(cs_ext_discount_amt) as "excess discount amount" -from - catalog_sales - ,item - ,date_dim -where -i_manufact_id = 29 -and i_item_sk = cs_item_sk -and d_date between '1999-01-07' and - (cast('1999-01-07' as date) + interval 90 day) -and d_date_sk = cs_sold_date_sk -and cs_ext_discount_amt - > ( - select - 1.3 * avg(cs_ext_discount_amt) - from - catalog_sales - ,date_dim - where - cs_item_sk = i_item_sk - and d_date between '1999-01-07' and - (cast('1999-01-07' as date) + interval 90 day) - and d_date_sk = cs_sold_date_sk - ) -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query33.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query33.sql deleted file mode 100644 index 6a53522cfa2360e..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query33.sql +++ /dev/null @@ -1,77 +0,0 @@ -explain shape plan - - - - -with ss as ( - select - i_manufact_id,sum(ss_ext_sales_price) total_sales - from - store_sales, - date_dim, - customer_address, - item - where - i_manufact_id in (select - i_manufact_id -from - item -where i_category in ('Home')) - and ss_item_sk = i_item_sk - and ss_sold_date_sk = d_date_sk - and d_year = 2002 - and d_moy = 1 - and ss_addr_sk = ca_address_sk - and ca_gmt_offset = -5 - group by i_manufact_id), - cs as ( - select - i_manufact_id,sum(cs_ext_sales_price) total_sales - from - catalog_sales, - date_dim, - customer_address, - item - where - i_manufact_id in (select - i_manufact_id -from - item -where i_category in ('Home')) - and cs_item_sk = i_item_sk - and cs_sold_date_sk = d_date_sk - and d_year = 2002 - and d_moy = 1 - and cs_bill_addr_sk = ca_address_sk - and ca_gmt_offset = -5 - group by i_manufact_id), - ws as ( - select - i_manufact_id,sum(ws_ext_sales_price) total_sales - from - web_sales, - date_dim, - customer_address, - item - where - i_manufact_id in (select - i_manufact_id -from - item -where i_category in ('Home')) - and ws_item_sk = i_item_sk - and ws_sold_date_sk = d_date_sk - and d_year = 2002 - and d_moy = 1 - and ws_bill_addr_sk = ca_address_sk - and ca_gmt_offset = -5 - group by i_manufact_id) - select i_manufact_id ,sum(total_sales) total_sales - from (select * from ss - union all - select * from cs - union all - select * from ws) tmp1 - group by i_manufact_id - order by total_sales -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query34.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query34.sql deleted file mode 100644 index 481ad1fa8852534..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query34.sql +++ /dev/null @@ -1,33 +0,0 @@ -explain shape plan - - - - -select c_last_name - ,c_first_name - ,c_salutation - ,c_preferred_cust_flag - ,ss_ticket_number - ,cnt from - (select ss_ticket_number - ,ss_customer_sk - ,count(*) cnt - from store_sales,date_dim,store,household_demographics - where store_sales.ss_sold_date_sk = date_dim.d_date_sk - and store_sales.ss_store_sk = store.s_store_sk - and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) - and (household_demographics.hd_buy_potential = '1001-5000' or - household_demographics.hd_buy_potential = '0-500') - and household_demographics.hd_vehicle_count > 0 - and (case when household_demographics.hd_vehicle_count > 0 - then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count - else null - end) > 1.2 - and date_dim.d_year in (1998,1998+1,1998+2) - and store.s_county in ('Ziebach County','Daviess County','Walker County','Richland County', - 'Barrow County','Franklin Parish','Williamson County','Luce County') - group by ss_ticket_number,ss_customer_sk) dn,customer - where ss_customer_sk = c_customer_sk - and cnt between 15 and 20 - order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query35.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query35.sql deleted file mode 100644 index 380d28b92464861..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query35.sql +++ /dev/null @@ -1,60 +0,0 @@ -explain shape plan - - - - -select - ca_state, - cd_gender, - cd_marital_status, - cd_dep_count, - count(*) cnt1, - max(cd_dep_count), - sum(cd_dep_count), - max(cd_dep_count), - cd_dep_employed_count, - count(*) cnt2, - max(cd_dep_employed_count), - sum(cd_dep_employed_count), - max(cd_dep_employed_count), - cd_dep_college_count, - count(*) cnt3, - max(cd_dep_college_count), - sum(cd_dep_college_count), - max(cd_dep_college_count) - from - customer c,customer_address ca,customer_demographics - where - c.c_current_addr_sk = ca.ca_address_sk and - cd_demo_sk = c.c_current_cdemo_sk and - exists (select * - from store_sales,date_dim - where c.c_customer_sk = ss_customer_sk and - ss_sold_date_sk = d_date_sk and - d_year = 2001 and - d_qoy < 4) and - (exists (select * - from web_sales,date_dim - where c.c_customer_sk = ws_bill_customer_sk and - ws_sold_date_sk = d_date_sk and - d_year = 2001 and - d_qoy < 4) or - exists (select * - from catalog_sales,date_dim - where c.c_customer_sk = cs_ship_customer_sk and - cs_sold_date_sk = d_date_sk and - d_year = 2001 and - d_qoy < 4)) - group by ca_state, - cd_gender, - cd_marital_status, - cd_dep_count, - cd_dep_employed_count, - cd_dep_college_count - order by ca_state, - cd_gender, - cd_marital_status, - cd_dep_count, - cd_dep_employed_count, - cd_dep_college_count - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query36.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query36.sql deleted file mode 100644 index 512bf3dc8c2f3d2..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query36.sql +++ /dev/null @@ -1,32 +0,0 @@ -explain shape plan - - - - -select - sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin - ,i_category - ,i_class - ,grouping(i_category)+grouping(i_class) as lochierarchy - ,rank() over ( - partition by grouping(i_category)+grouping(i_class), - case when grouping(i_class) = 0 then i_category end - order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent - from - store_sales - ,date_dim d1 - ,item - ,store - where - d1.d_year = 2002 - and d1.d_date_sk = ss_sold_date_sk - and i_item_sk = ss_item_sk - and s_store_sk = ss_store_sk - and s_state in ('SD','TN','GA','SC', - 'MO','AL','MI','OH') - group by rollup(i_category,i_class) - order by - lochierarchy desc - ,case when lochierarchy = 0 then i_category end - ,rank_within_parent - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query37.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query37.sql deleted file mode 100644 index 4dcc03d28966526..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query37.sql +++ /dev/null @@ -1,19 +0,0 @@ -explain shape plan - - - - -select i_item_id - ,i_item_desc - ,i_current_price - from item, inventory, date_dim, catalog_sales - where i_current_price between 45 and 45 + 30 - and inv_item_sk = i_item_sk - and d_date_sk=inv_date_sk - and d_date between cast('1999-02-21' as date) and (cast('1999-02-21' as date) + interval 60 day) - and i_manufact_id in (856,707,1000,747) - and inv_quantity_on_hand between 100 and 500 - and cs_item_sk = i_item_sk - group by i_item_id,i_item_desc,i_current_price - order by i_item_id - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query38.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query38.sql deleted file mode 100644 index 179d248918b27e0..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query38.sql +++ /dev/null @@ -1,25 +0,0 @@ -explain shape plan - - - - -select count(*) from ( - select distinct c_last_name, c_first_name, d_date - from store_sales, date_dim, customer - where store_sales.ss_sold_date_sk = date_dim.d_date_sk - and store_sales.ss_customer_sk = customer.c_customer_sk - and d_month_seq between 1183 and 1183 + 11 - intersect - select distinct c_last_name, c_first_name, d_date - from catalog_sales, date_dim, customer - where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk - and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk - and d_month_seq between 1183 and 1183 + 11 - intersect - select distinct c_last_name, c_first_name, d_date - from web_sales, date_dim, customer - where web_sales.ws_sold_date_sk = date_dim.d_date_sk - and web_sales.ws_bill_customer_sk = customer.c_customer_sk - and d_month_seq between 1183 and 1183 + 11 -) hot_cust -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query39.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query39.sql deleted file mode 100644 index 68f39fda68e9527..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query39.sql +++ /dev/null @@ -1,30 +0,0 @@ -explain shape plan - - - - -with inv as -(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy - ,stdev,mean, case mean when 0 then null else stdev/mean end cov - from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy - ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean - from inventory - ,item - ,warehouse - ,date_dim - where inv_item_sk = i_item_sk - and inv_warehouse_sk = w_warehouse_sk - and inv_date_sk = d_date_sk - and d_year =1998 - group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo - where case mean when 0 then 0 else stdev/mean end > 1) -select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov - ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov -from inv inv1,inv inv2 -where inv1.i_item_sk = inv2.i_item_sk - and inv1.w_warehouse_sk = inv2.w_warehouse_sk - and inv1.d_moy=1 - and inv2.d_moy=1+1 -order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov - ,inv2.d_moy,inv2.mean, inv2.cov -; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query40.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query40.sql deleted file mode 100644 index f54199c1ae5c905..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query40.sql +++ /dev/null @@ -1,29 +0,0 @@ -explain shape plan - - - -select - w_state - ,i_item_id - ,sum(case when (cast(d_date as date) < cast ('2001-04-02' as date)) - then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before - ,sum(case when (cast(d_date as date) >= cast ('2001-04-02' as date)) - then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after - from - catalog_sales left outer join catalog_returns on - (cs_order_number = cr_order_number - and cs_item_sk = cr_item_sk) - ,warehouse - ,item - ,date_dim - where - i_current_price between 0.99 and 1.49 - and i_item_sk = cs_item_sk - and cs_warehouse_sk = w_warehouse_sk - and cs_sold_date_sk = d_date_sk - and d_date between (cast ('2001-04-02' as date) - interval 30 day) - and (cast ('2001-04-02' as date) + interval 30 day) - group by - w_state,i_item_id - order by w_state,i_item_id -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query41.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query41.sql deleted file mode 100644 index ef6b712c2779458..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query41.sql +++ /dev/null @@ -1,54 +0,0 @@ -explain shape plan - - - - -select distinct(i_product_name) - from item i1 - where i_manufact_id between 748 and 748+40 - and (select count(*) as item_cnt - from item - where (i_manufact = i1.i_manufact and - ((i_category = 'Women' and - (i_color = 'gainsboro' or i_color = 'aquamarine') and - (i_units = 'Ounce' or i_units = 'Dozen') and - (i_size = 'medium' or i_size = 'economy') - ) or - (i_category = 'Women' and - (i_color = 'chiffon' or i_color = 'violet') and - (i_units = 'Ton' or i_units = 'Pound') and - (i_size = 'extra large' or i_size = 'small') - ) or - (i_category = 'Men' and - (i_color = 'chartreuse' or i_color = 'blue') and - (i_units = 'Each' or i_units = 'Oz') and - (i_size = 'N/A' or i_size = 'large') - ) or - (i_category = 'Men' and - (i_color = 'tan' or i_color = 'dodger') and - (i_units = 'Bunch' or i_units = 'Tsp') and - (i_size = 'medium' or i_size = 'economy') - ))) or - (i_manufact = i1.i_manufact and - ((i_category = 'Women' and - (i_color = 'blanched' or i_color = 'tomato') and - (i_units = 'Tbl' or i_units = 'Case') and - (i_size = 'medium' or i_size = 'economy') - ) or - (i_category = 'Women' and - (i_color = 'almond' or i_color = 'lime') and - (i_units = 'Box' or i_units = 'Dram') and - (i_size = 'extra large' or i_size = 'small') - ) or - (i_category = 'Men' and - (i_color = 'peru' or i_color = 'saddle') and - (i_units = 'Pallet' or i_units = 'Gram') and - (i_size = 'N/A' or i_size = 'large') - ) or - (i_category = 'Men' and - (i_color = 'indian' or i_color = 'spring') and - (i_units = 'Unknown' or i_units = 'Carton') and - (i_size = 'medium' or i_size = 'economy') - )))) > 0 - order by i_product_name - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query42.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query42.sql deleted file mode 100644 index 981e187c4b5524b..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query42.sql +++ /dev/null @@ -1,24 +0,0 @@ -explain shape plan - - - - -select dt.d_year - ,item.i_category_id - ,item.i_category - ,sum(ss_ext_sales_price) - from date_dim dt - ,store_sales - ,item - where dt.d_date_sk = store_sales.ss_sold_date_sk - and store_sales.ss_item_sk = item.i_item_sk - and item.i_manager_id = 1 - and dt.d_moy=11 - and dt.d_year=2002 - group by dt.d_year - ,item.i_category_id - ,item.i_category - order by sum(ss_ext_sales_price) desc,dt.d_year - ,item.i_category_id - ,item.i_category -limit 100 ; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query43.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query43.sql deleted file mode 100644 index 0576ea5fe604b13..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query43.sql +++ /dev/null @@ -1,21 +0,0 @@ -explain shape plan - - - - -select s_store_name, s_store_id, - sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, - sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, - sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, - sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, - sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, - sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, - sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales - from date_dim, store_sales, store - where d_date_sk = ss_sold_date_sk and - s_store_sk = ss_store_sk and - s_gmt_offset = -5 and - d_year = 2000 - group by s_store_name, s_store_id - order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query44.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query44.sql deleted file mode 100644 index fc568127a2a32f2..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query44.sql +++ /dev/null @@ -1,37 +0,0 @@ -explain shape plan - - - - -select asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing -from(select * - from (select item_sk,rank() over (order by rank_col asc) rnk - from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col - from store_sales ss1 - where ss_store_sk = 146 - group by ss_item_sk - having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col - from store_sales - where ss_store_sk = 146 - and ss_addr_sk is null - group by ss_store_sk))V1)V11 - where rnk < 11) asceding, - (select * - from (select item_sk,rank() over (order by rank_col desc) rnk - from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col - from store_sales ss1 - where ss_store_sk = 146 - group by ss_item_sk - having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col - from store_sales - where ss_store_sk = 146 - and ss_addr_sk is null - group by ss_store_sk))V2)V21 - where rnk < 11) descending, -item i1, -item i2 -where asceding.rnk = descending.rnk - and i1.i_item_sk=asceding.item_sk - and i2.i_item_sk=descending.item_sk -order by asceding.rnk -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query45.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query45.sql deleted file mode 100644 index 58d612bf99bdf8f..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query45.sql +++ /dev/null @@ -1,22 +0,0 @@ -explain shape plan - - - - -select ca_zip, ca_city, sum(ws_sales_price) - from web_sales, customer, customer_address, date_dim, item - where ws_bill_customer_sk = c_customer_sk - and c_current_addr_sk = ca_address_sk - and ws_item_sk = i_item_sk - and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') - or - i_item_id in (select i_item_id - from item - where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) - ) - ) - and ws_sold_date_sk = d_date_sk - and d_qoy = 2 and d_year = 2000 - group by ca_zip, ca_city - order by ca_zip, ca_city - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query46.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query46.sql deleted file mode 100644 index ec1e80e449bf77e..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query46.sql +++ /dev/null @@ -1,37 +0,0 @@ -explain shape plan - - - - -select c_last_name - ,c_first_name - ,ca_city - ,bought_city - ,ss_ticket_number - ,amt,profit - from - (select ss_ticket_number - ,ss_customer_sk - ,ca_city bought_city - ,sum(ss_coupon_amt) amt - ,sum(ss_net_profit) profit - from store_sales,date_dim,store,household_demographics,customer_address - where store_sales.ss_sold_date_sk = date_dim.d_date_sk - and store_sales.ss_store_sk = store.s_store_sk - and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - and store_sales.ss_addr_sk = customer_address.ca_address_sk - and (household_demographics.hd_dep_count = 6 or - household_demographics.hd_vehicle_count= 0) - and date_dim.d_dow in (6,0) - and date_dim.d_year in (1999,1999+1,1999+2) - and store.s_city in ('Five Points','Centerville','Oak Grove','Fairview','Liberty') - group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr - where ss_customer_sk = c_customer_sk - and customer.c_current_addr_sk = current_addr.ca_address_sk - and current_addr.ca_city <> bought_city - order by c_last_name - ,c_first_name - ,ca_city - ,bought_city - ,ss_ticket_number - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query47.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query47.sql deleted file mode 100644 index 8ef537ee32ce7e5..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query47.sql +++ /dev/null @@ -1,51 +0,0 @@ -explain shape plan - - -with v1 as( - select i_category, i_brand, - s_store_name, s_company_name, - d_year, d_moy, - sum(ss_sales_price) sum_sales, - avg(sum(ss_sales_price)) over - (partition by i_category, i_brand, - s_store_name, s_company_name, d_year) - avg_monthly_sales, - rank() over - (partition by i_category, i_brand, - s_store_name, s_company_name - order by d_year, d_moy) rn - from item, store_sales, date_dim, store - where ss_item_sk = i_item_sk and - ss_sold_date_sk = d_date_sk and - ss_store_sk = s_store_sk and - ( - d_year = 2001 or - ( d_year = 2001-1 and d_moy =12) or - ( d_year = 2001+1 and d_moy =1) - ) - group by i_category, i_brand, - s_store_name, s_company_name, - d_year, d_moy), - v2 as( - select v1.s_store_name - ,v1.d_year - ,v1.avg_monthly_sales - ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum - from v1, v1 v1_lag, v1 v1_lead - where v1.i_category = v1_lag.i_category and - v1.i_category = v1_lead.i_category and - v1.i_brand = v1_lag.i_brand and - v1.i_brand = v1_lead.i_brand and - v1.s_store_name = v1_lag.s_store_name and - v1.s_store_name = v1_lead.s_store_name and - v1.s_company_name = v1_lag.s_company_name and - v1.s_company_name = v1_lead.s_company_name and - v1.rn = v1_lag.rn + 1 and - v1.rn = v1_lead.rn - 1) - select * - from v2 - where d_year = 2001 and - avg_monthly_sales > 0 and - case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 - order by sum_sales - avg_monthly_sales, nsum - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query48.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query48.sql deleted file mode 100644 index 2a16fb07c70f761..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query48.sql +++ /dev/null @@ -1,69 +0,0 @@ -explain shape plan - - - - -select sum (ss_quantity) - from store_sales, store, customer_demographics, customer_address, date_dim - where s_store_sk = ss_store_sk - and ss_sold_date_sk = d_date_sk and d_year = 1999 - and - ( - ( - cd_demo_sk = ss_cdemo_sk - and - cd_marital_status = 'U' - and - cd_education_status = 'Primary' - and - ss_sales_price between 100.00 and 150.00 - ) - or - ( - cd_demo_sk = ss_cdemo_sk - and - cd_marital_status = 'W' - and - cd_education_status = 'College' - and - ss_sales_price between 50.00 and 100.00 - ) - or - ( - cd_demo_sk = ss_cdemo_sk - and - cd_marital_status = 'D' - and - cd_education_status = '2 yr Degree' - and - ss_sales_price between 150.00 and 200.00 - ) - ) - and - ( - ( - ss_addr_sk = ca_address_sk - and - ca_country = 'United States' - and - ca_state in ('MD', 'MN', 'IA') - and ss_net_profit between 0 and 2000 - ) - or - (ss_addr_sk = ca_address_sk - and - ca_country = 'United States' - and - ca_state in ('VA', 'IL', 'TX') - and ss_net_profit between 150 and 3000 - ) - or - (ss_addr_sk = ca_address_sk - and - ca_country = 'United States' - and - ca_state in ('MI', 'WI', 'IN') - and ss_net_profit between 50 and 25000 - ) - ) -; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query50.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query50.sql deleted file mode 100644 index 61eb50bcb7f9429..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query50.sql +++ /dev/null @@ -1,61 +0,0 @@ -explain shape plan - - - - -select - s_store_name - ,s_company_id - ,s_street_number - ,s_street_name - ,s_street_type - ,s_suite_number - ,s_city - ,s_county - ,s_state - ,s_zip - ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" - ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and - (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" - ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and - (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" - ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and - (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" - ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 120) then 1 else 0 end) as ">120 days" -from - store_sales - ,store_returns - ,store - ,date_dim d1 - ,date_dim d2 -where - d2.d_year = 2001 -and d2.d_moy = 8 -and ss_ticket_number = sr_ticket_number -and ss_item_sk = sr_item_sk -and ss_sold_date_sk = d1.d_date_sk -and sr_returned_date_sk = d2.d_date_sk -and ss_customer_sk = sr_customer_sk -and ss_store_sk = s_store_sk -group by - s_store_name - ,s_company_id - ,s_street_number - ,s_street_name - ,s_street_type - ,s_suite_number - ,s_city - ,s_county - ,s_state - ,s_zip -order by s_store_name - ,s_company_id - ,s_street_number - ,s_street_name - ,s_street_type - ,s_suite_number - ,s_city - ,s_county - ,s_state - ,s_zip -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query51.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query51.sql deleted file mode 100644 index 0cbfc7ace2708d2..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query51.sql +++ /dev/null @@ -1,47 +0,0 @@ -explain shape plan - - - - -WITH web_v1 as ( -select - ws_item_sk item_sk, d_date, - sum(sum(ws_sales_price)) - over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales -from web_sales - ,date_dim -where ws_sold_date_sk=d_date_sk - and d_month_seq between 1216 and 1216+11 - and ws_item_sk is not NULL -group by ws_item_sk, d_date), -store_v1 as ( -select - ss_item_sk item_sk, d_date, - sum(sum(ss_sales_price)) - over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales -from store_sales - ,date_dim -where ss_sold_date_sk=d_date_sk - and d_month_seq between 1216 and 1216+11 - and ss_item_sk is not NULL -group by ss_item_sk, d_date) - select * -from (select item_sk - ,d_date - ,web_sales - ,store_sales - ,max(web_sales) - over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative - ,max(store_sales) - over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative - from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk - ,case when web.d_date is not null then web.d_date else store.d_date end d_date - ,web.cume_sales web_sales - ,store.cume_sales store_sales - from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk - and web.d_date = store.d_date) - )x )y -where web_cumulative > store_cumulative -order by item_sk - ,d_date -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query52.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query52.sql deleted file mode 100644 index 70c3e06a7957db2..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query52.sql +++ /dev/null @@ -1,24 +0,0 @@ -explain shape plan - - - - -select dt.d_year - ,item.i_brand_id brand_id - ,item.i_brand brand - ,sum(ss_ext_sales_price) ext_price - from date_dim dt - ,store_sales - ,item - where dt.d_date_sk = store_sales.ss_sold_date_sk - and store_sales.ss_item_sk = item.i_item_sk - and item.i_manager_id = 1 - and dt.d_moy=12 - and dt.d_year=2002 - group by dt.d_year - ,item.i_brand - ,item.i_brand_id - order by dt.d_year - ,ext_price desc - ,brand_id -limit 100 ; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query53.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query53.sql deleted file mode 100644 index 4119858b3e2418c..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query53.sql +++ /dev/null @@ -1,30 +0,0 @@ -explain shape plan - - - - -select * from -(select i_manufact_id, -sum(ss_sales_price) sum_sales, -avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales -from item, store_sales, date_dim, store -where ss_item_sk = i_item_sk and -ss_sold_date_sk = d_date_sk and -ss_store_sk = s_store_sk and -d_month_seq in (1200,1200+1,1200+2,1200+3,1200+4,1200+5,1200+6,1200+7,1200+8,1200+9,1200+10,1200+11) and -((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'))) -group by i_manufact_id, d_qoy ) tmp1 -where case when avg_quarterly_sales > 0 - then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales - else null end > 0.1 -order by avg_quarterly_sales, - sum_sales, - i_manufact_id -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query54.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query54.sql deleted file mode 100644 index a4258ef7d77a186..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query54.sql +++ /dev/null @@ -1,58 +0,0 @@ -explain shape plan - - - - -with my_customers as ( - select distinct c_customer_sk - , c_current_addr_sk - from - ( select cs_sold_date_sk sold_date_sk, - cs_bill_customer_sk customer_sk, - cs_item_sk item_sk - from catalog_sales - union all - select ws_sold_date_sk sold_date_sk, - ws_bill_customer_sk customer_sk, - ws_item_sk item_sk - from web_sales - ) cs_or_ws_sales, - item, - date_dim, - customer - where sold_date_sk = d_date_sk - and item_sk = i_item_sk - and i_category = 'Women' - and i_class = 'maternity' - and c_customer_sk = cs_or_ws_sales.customer_sk - and d_moy = 5 - and d_year = 1998 - ) - , my_revenue as ( - select c_customer_sk, - sum(ss_ext_sales_price) as revenue - from my_customers, - store_sales, - customer_address, - store, - date_dim - where c_current_addr_sk = ca_address_sk - and ca_county = s_county - and ca_state = s_state - and ss_sold_date_sk = d_date_sk - and c_customer_sk = ss_customer_sk - and d_month_seq between (select distinct d_month_seq+1 - from date_dim where d_year = 1998 and d_moy = 5) - and (select distinct d_month_seq+3 - from date_dim where d_year = 1998 and d_moy = 5) - group by c_customer_sk - ) - , segments as - (select cast((revenue/50) as int) as segment - from my_revenue - ) - select segment, count(*) as num_customers, segment*50 as segment_base - from segments - group by segment - order by segment, num_customers - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query55.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query55.sql deleted file mode 100644 index 5db349671964899..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query55.sql +++ /dev/null @@ -1,16 +0,0 @@ -explain shape plan - - - - -select i_brand_id brand_id, i_brand brand, - sum(ss_ext_sales_price) ext_price - from date_dim, store_sales, item - where d_date_sk = ss_sold_date_sk - and ss_item_sk = i_item_sk - and i_manager_id=100 - and d_moy=12 - and d_year=2000 - group by i_brand, i_brand_id - order by ext_price desc, i_brand_id -limit 100 ; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query56.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query56.sql deleted file mode 100644 index 157a6ee307e9aab..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query56.sql +++ /dev/null @@ -1,71 +0,0 @@ -explain shape plan - - - - -with ss as ( - select i_item_id,sum(ss_ext_sales_price) total_sales - from - store_sales, - date_dim, - customer_address, - item - where i_item_id in (select - i_item_id -from item -where i_color in ('powder','green','cyan')) - and ss_item_sk = i_item_sk - and ss_sold_date_sk = d_date_sk - and d_year = 2000 - and d_moy = 2 - and ss_addr_sk = ca_address_sk - and ca_gmt_offset = -6 - group by i_item_id), - cs as ( - select i_item_id,sum(cs_ext_sales_price) total_sales - from - catalog_sales, - date_dim, - customer_address, - item - where - i_item_id in (select - i_item_id -from item -where i_color in ('powder','green','cyan')) - and cs_item_sk = i_item_sk - and cs_sold_date_sk = d_date_sk - and d_year = 2000 - and d_moy = 2 - and cs_bill_addr_sk = ca_address_sk - and ca_gmt_offset = -6 - group by i_item_id), - ws as ( - select i_item_id,sum(ws_ext_sales_price) total_sales - from - web_sales, - date_dim, - customer_address, - item - where - i_item_id in (select - i_item_id -from item -where i_color in ('powder','green','cyan')) - and ws_item_sk = i_item_sk - and ws_sold_date_sk = d_date_sk - and d_year = 2000 - and d_moy = 2 - and ws_bill_addr_sk = ca_address_sk - and ca_gmt_offset = -6 - group by i_item_id) - select i_item_id ,sum(total_sales) total_sales - from (select * from ss - union all - select * from cs - union all - select * from ws) tmp1 - group by i_item_id - order by total_sales, - i_item_id - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query57.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query57.sql deleted file mode 100644 index e6bc24346daaf07..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query57.sql +++ /dev/null @@ -1,49 +0,0 @@ -explain shape plan - - - -with v1 as( - select i_category, i_brand, - cc_name, - d_year, d_moy, - sum(cs_sales_price) sum_sales, - avg(sum(cs_sales_price)) over - (partition by i_category, i_brand, - cc_name, d_year) - avg_monthly_sales, - rank() over - (partition by i_category, i_brand, - cc_name - order by d_year, d_moy) rn - from item, catalog_sales, date_dim, call_center - where cs_item_sk = i_item_sk and - cs_sold_date_sk = d_date_sk and - cc_call_center_sk= cs_call_center_sk and - ( - d_year = 1999 or - ( d_year = 1999-1 and d_moy =12) or - ( d_year = 1999+1 and d_moy =1) - ) - group by i_category, i_brand, - cc_name , d_year, d_moy), - v2 as( - select v1.i_brand - ,v1.d_year - ,v1.avg_monthly_sales - ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum - from v1, v1 v1_lag, v1 v1_lead - where v1.i_category = v1_lag.i_category and - v1.i_category = v1_lead.i_category and - v1.i_brand = v1_lag.i_brand and - v1.i_brand = v1_lead.i_brand and - v1. cc_name = v1_lag. cc_name and - v1. cc_name = v1_lead. cc_name and - v1.rn = v1_lag.rn + 1 and - v1.rn = v1_lead.rn - 1) - select * - from v2 - where d_year = 1999 and - avg_monthly_sales > 0 and - case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 - order by sum_sales - avg_monthly_sales, nsum - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query58.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query58.sql deleted file mode 100644 index 53cc8f4ad65526b..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query58.sql +++ /dev/null @@ -1,67 +0,0 @@ -explain shape plan - - - - -with ss_items as - (select i_item_id item_id - ,sum(ss_ext_sales_price) ss_item_rev - from store_sales - ,item - ,date_dim - where ss_item_sk = i_item_sk - and d_date in (select d_date - from date_dim - where d_week_seq = (select d_week_seq - from date_dim - where d_date = '2001-03-24')) - and ss_sold_date_sk = d_date_sk - group by i_item_id), - cs_items as - (select i_item_id item_id - ,sum(cs_ext_sales_price) cs_item_rev - from catalog_sales - ,item - ,date_dim - where cs_item_sk = i_item_sk - and d_date in (select d_date - from date_dim - where d_week_seq = (select d_week_seq - from date_dim - where d_date = '2001-03-24')) - and cs_sold_date_sk = d_date_sk - group by i_item_id), - ws_items as - (select i_item_id item_id - ,sum(ws_ext_sales_price) ws_item_rev - from web_sales - ,item - ,date_dim - where ws_item_sk = i_item_sk - and d_date in (select d_date - from date_dim - where d_week_seq =(select d_week_seq - from date_dim - where d_date = '2001-03-24')) - and ws_sold_date_sk = d_date_sk - group by i_item_id) - select ss_items.item_id - ,ss_item_rev - ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev - ,cs_item_rev - ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev - ,ws_item_rev - ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev - ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average - from ss_items,cs_items,ws_items - where ss_items.item_id=cs_items.item_id - and ss_items.item_id=ws_items.item_id - and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev - and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev - and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev - and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev - and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev - and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev - order by item_id - ,ss_item_rev - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query59.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query59.sql deleted file mode 100644 index 97d49c37cfb76e8..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query59.sql +++ /dev/null @@ -1,45 +0,0 @@ -explain shape plan - - - -with wss as - (select d_week_seq, - ss_store_sk, - sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, - sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, - sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, - sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, - sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, - sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, - sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales - from store_sales,date_dim - where d_date_sk = ss_sold_date_sk - group by d_week_seq,ss_store_sk - ) - select s_store_name1,s_store_id1,d_week_seq1 - ,sun_sales1/sun_sales2,mon_sales1/mon_sales2 - ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2 - ,fri_sales1/fri_sales2,sat_sales1/sat_sales2 - from - (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1 - ,s_store_id s_store_id1,sun_sales sun_sales1 - ,mon_sales mon_sales1,tue_sales tue_sales1 - ,wed_sales wed_sales1,thu_sales thu_sales1 - ,fri_sales fri_sales1,sat_sales sat_sales1 - from wss,store,date_dim d - where d.d_week_seq = wss.d_week_seq and - ss_store_sk = s_store_sk and - d_month_seq between 1196 and 1196 + 11) y, - (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2 - ,s_store_id s_store_id2,sun_sales sun_sales2 - ,mon_sales mon_sales2,tue_sales tue_sales2 - ,wed_sales wed_sales2,thu_sales thu_sales2 - ,fri_sales fri_sales2,sat_sales sat_sales2 - from wss,store,date_dim d - where d.d_week_seq = wss.d_week_seq and - ss_store_sk = s_store_sk and - d_month_seq between 1196+ 12 and 1196 + 23) x - where s_store_id1=s_store_id2 - and d_week_seq1=d_week_seq2-52 - order by s_store_name1,s_store_id1,d_week_seq1 -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query6.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query6.sql deleted file mode 100644 index 239a683f48b7159..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query6.sql +++ /dev/null @@ -1,28 +0,0 @@ -explain shape plan - - - - -select a.ca_state state, count(*) cnt - from customer_address a - ,customer c - ,store_sales s - ,date_dim d - ,item i - where a.ca_address_sk = c.c_current_addr_sk - and c.c_customer_sk = s.ss_customer_sk - and s.ss_sold_date_sk = d.d_date_sk - and s.ss_item_sk = i.i_item_sk - and d.d_month_seq = - (select distinct (d_month_seq) - from date_dim - where d_year = 2002 - and d_moy = 3 ) - and i.i_current_price > 1.2 * - (select avg(j.i_current_price) - from item j - where j.i_category = i.i_category) - group by a.ca_state - having count(*) >= 10 - order by cnt, a.ca_state - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query60.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query60.sql deleted file mode 100644 index 61b9d3bf0b6d9d0..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query60.sql +++ /dev/null @@ -1,80 +0,0 @@ -explain shape plan - - - - -with ss as ( - select - i_item_id,sum(ss_ext_sales_price) total_sales - from - store_sales, - date_dim, - customer_address, - item - where - i_item_id in (select - i_item_id -from - item -where i_category in ('Children')) - and ss_item_sk = i_item_sk - and ss_sold_date_sk = d_date_sk - and d_year = 2000 - and d_moy = 8 - and ss_addr_sk = ca_address_sk - and ca_gmt_offset = -7 - group by i_item_id), - cs as ( - select - i_item_id,sum(cs_ext_sales_price) total_sales - from - catalog_sales, - date_dim, - customer_address, - item - where - i_item_id in (select - i_item_id -from - item -where i_category in ('Children')) - and cs_item_sk = i_item_sk - and cs_sold_date_sk = d_date_sk - and d_year = 2000 - and d_moy = 8 - and cs_bill_addr_sk = ca_address_sk - and ca_gmt_offset = -7 - group by i_item_id), - ws as ( - select - i_item_id,sum(ws_ext_sales_price) total_sales - from - web_sales, - date_dim, - customer_address, - item - where - i_item_id in (select - i_item_id -from - item -where i_category in ('Children')) - and ws_item_sk = i_item_sk - and ws_sold_date_sk = d_date_sk - and d_year = 2000 - and d_moy = 8 - and ws_bill_addr_sk = ca_address_sk - and ca_gmt_offset = -7 - group by i_item_id) - select - i_item_id -,sum(total_sales) total_sales - from (select * from ss - union all - select * from cs - union all - select * from ws) tmp1 - group by i_item_id - order by i_item_id - ,total_sales - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query61.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query61.sql deleted file mode 100644 index 5e745051fa2374e..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query61.sql +++ /dev/null @@ -1,46 +0,0 @@ -explain shape plan - - - - -select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100 -from - (select sum(ss_ext_sales_price) promotions - from store_sales - ,store - ,promotion - ,date_dim - ,customer - ,customer_address - ,item - where ss_sold_date_sk = d_date_sk - and ss_store_sk = s_store_sk - and ss_promo_sk = p_promo_sk - and ss_customer_sk= c_customer_sk - and ca_address_sk = c_current_addr_sk - and ss_item_sk = i_item_sk - and ca_gmt_offset = -7 - and i_category = 'Jewelry' - and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') - and s_gmt_offset = -7 - and d_year = 1999 - and d_moy = 11) promotional_sales, - (select sum(ss_ext_sales_price) total - from store_sales - ,store - ,date_dim - ,customer - ,customer_address - ,item - where ss_sold_date_sk = d_date_sk - and ss_store_sk = s_store_sk - and ss_customer_sk= c_customer_sk - and ca_address_sk = c_current_addr_sk - and ss_item_sk = i_item_sk - and ca_gmt_offset = -7 - and i_category = 'Jewelry' - and s_gmt_offset = -7 - and d_year = 1999 - and d_moy = 11) all_sales -order by promotions, total -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query62.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query62.sql deleted file mode 100644 index 9d129c5b296e735..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query62.sql +++ /dev/null @@ -1,37 +0,0 @@ -explain shape plan - - - - -select - substr(w_warehouse_name,1,20) - ,sm_type - ,web_name - ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" - ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and - (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" - ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and - (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" - ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and - (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" - ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as ">120 days" -from - web_sales - ,warehouse - ,ship_mode - ,web_site - ,date_dim -where - d_month_seq between 1194 and 1194 + 11 -and ws_ship_date_sk = d_date_sk -and ws_warehouse_sk = w_warehouse_sk -and ws_ship_mode_sk = sm_ship_mode_sk -and ws_web_site_sk = web_site_sk -group by - substr(w_warehouse_name,1,20) - ,sm_type - ,web_name -order by substr(w_warehouse_name,1,20) - ,sm_type - ,web_name -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query63.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query63.sql deleted file mode 100644 index dd0523d289c1b8f..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query63.sql +++ /dev/null @@ -1,31 +0,0 @@ -explain shape plan - - - - -select * -from (select i_manager_id - ,sum(ss_sales_price) sum_sales - ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales - from item - ,store_sales - ,date_dim - ,store - where ss_item_sk = i_item_sk - and ss_sold_date_sk = d_date_sk - and ss_store_sk = s_store_sk - and d_month_seq in (1181,1181+1,1181+2,1181+3,1181+4,1181+5,1181+6,1181+7,1181+8,1181+9,1181+10,1181+11) - and (( 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'))) -group by i_manager_id, d_moy) tmp1 -where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 -order by i_manager_id - ,avg_monthly_sales - ,sum_sales -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query64.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query64.sql deleted file mode 100644 index 4d2f50d97d0223e..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query64.sql +++ /dev/null @@ -1,123 +0,0 @@ - explain shape plan - - - - - with cs_ui as - (select cs_item_sk - ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund - from catalog_sales - ,catalog_returns - where cs_item_sk = cr_item_sk - and cs_order_number = cr_order_number - group by cs_item_sk - having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)), - cross_sales as - (select i_product_name product_name - ,i_item_sk item_sk - ,s_store_name store_name - ,s_zip store_zip - ,ad1.ca_street_number b_street_number - ,ad1.ca_street_name b_street_name - ,ad1.ca_city b_city - ,ad1.ca_zip b_zip - ,ad2.ca_street_number c_street_number - ,ad2.ca_street_name c_street_name - ,ad2.ca_city c_city - ,ad2.ca_zip c_zip - ,d1.d_year as syear - ,d2.d_year as fsyear - ,d3.d_year s2year - ,count(*) cnt - ,sum(ss_wholesale_cost) s1 - ,sum(ss_list_price) s2 - ,sum(ss_coupon_amt) s3 - FROM store_sales - ,store_returns - ,cs_ui - ,date_dim d1 - ,date_dim d2 - ,date_dim d3 - ,store - ,customer - ,customer_demographics cd1 - ,customer_demographics cd2 - ,promotion - ,household_demographics hd1 - ,household_demographics hd2 - ,customer_address ad1 - ,customer_address ad2 - ,income_band ib1 - ,income_band ib2 - ,item - WHERE ss_store_sk = s_store_sk AND - ss_sold_date_sk = d1.d_date_sk AND - ss_customer_sk = c_customer_sk AND - ss_cdemo_sk= cd1.cd_demo_sk AND - ss_hdemo_sk = hd1.hd_demo_sk AND - ss_addr_sk = ad1.ca_address_sk and - ss_item_sk = i_item_sk and - ss_item_sk = sr_item_sk and - ss_ticket_number = sr_ticket_number and - ss_item_sk = cs_ui.cs_item_sk and - c_current_cdemo_sk = cd2.cd_demo_sk AND - c_current_hdemo_sk = hd2.hd_demo_sk AND - c_current_addr_sk = ad2.ca_address_sk and - c_first_sales_date_sk = d2.d_date_sk and - c_first_shipto_date_sk = d3.d_date_sk and - ss_promo_sk = p_promo_sk and - hd1.hd_income_band_sk = ib1.ib_income_band_sk and - hd2.hd_income_band_sk = ib2.ib_income_band_sk and - cd1.cd_marital_status <> cd2.cd_marital_status and - i_color in ('blanched','medium','brown','chocolate','burlywood','drab') and - i_current_price between 23 and 23 + 10 and - i_current_price between 23 + 1 and 23 + 15 - group by i_product_name - ,i_item_sk - ,s_store_name - ,s_zip - ,ad1.ca_street_number - ,ad1.ca_street_name - ,ad1.ca_city - ,ad1.ca_zip - ,ad2.ca_street_number - ,ad2.ca_street_name - ,ad2.ca_city - ,ad2.ca_zip - ,d1.d_year - ,d2.d_year - ,d3.d_year - ) - select cs1.product_name - ,cs1.store_name - ,cs1.store_zip - ,cs1.b_street_number - ,cs1.b_street_name - ,cs1.b_city - ,cs1.b_zip - ,cs1.c_street_number - ,cs1.c_street_name - ,cs1.c_city - ,cs1.c_zip - ,cs1.syear - ,cs1.cnt - ,cs1.s1 as s11 - ,cs1.s2 as s21 - ,cs1.s3 as s31 - ,cs2.s1 as s12 - ,cs2.s2 as s22 - ,cs2.s3 as s32 - ,cs2.syear - ,cs2.cnt - from cross_sales cs1,cross_sales cs2 - where cs1.item_sk=cs2.item_sk and - cs1.syear = 2001 and - cs2.syear = 2001 + 1 and - cs2.cnt <= cs1.cnt and - cs1.store_name = cs2.store_name and - cs1.store_zip = cs2.store_zip - order by cs1.product_name - ,cs1.store_name - ,cs2.cnt - ,cs1.s1 - ,cs2.s1; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query65.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query65.sql deleted file mode 100644 index 405022dc8d17850..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query65.sql +++ /dev/null @@ -1,31 +0,0 @@ -explain shape plan - - - - -select - s_store_name, - i_item_desc, - sc.revenue, - i_current_price, - i_wholesale_cost, - i_brand - from store, item, - (select ss_store_sk, avg(revenue) as ave - from - (select ss_store_sk, ss_item_sk, - sum(ss_sales_price) as revenue - from store_sales, date_dim - where ss_sold_date_sk = d_date_sk and d_month_seq between 1221 and 1221+11 - group by ss_store_sk, ss_item_sk) sa - group by ss_store_sk) sb, - (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue - from store_sales, date_dim - where ss_sold_date_sk = d_date_sk and d_month_seq between 1221 and 1221+11 - group by ss_store_sk, ss_item_sk) sc - where sb.ss_store_sk = sc.ss_store_sk and - sc.revenue <= 0.1 * sb.ave and - s_store_sk = sc.ss_store_sk and - i_item_sk = sc.ss_item_sk - order by s_store_name, i_item_desc -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query67.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query67.sql deleted file mode 100644 index 80d657d91ed8703..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query67.sql +++ /dev/null @@ -1,46 +0,0 @@ -explain shape plan - - - - -select * -from (select i_category - ,i_class - ,i_brand - ,i_product_name - ,d_year - ,d_qoy - ,d_moy - ,s_store_id - ,sumsales - ,rank() over (partition by i_category order by sumsales desc) rk - from (select i_category - ,i_class - ,i_brand - ,i_product_name - ,d_year - ,d_qoy - ,d_moy - ,s_store_id - ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales - from store_sales - ,date_dim - ,store - ,item - where ss_sold_date_sk=d_date_sk - and ss_item_sk=i_item_sk - and ss_store_sk = s_store_sk - and d_month_seq between 1206 and 1206+11 - group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2 -where rk <= 100 -order by i_category - ,i_class - ,i_brand - ,i_product_name - ,d_year - ,d_qoy - ,d_moy - ,s_store_id - ,sumsales - ,rk -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query68.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query68.sql deleted file mode 100644 index a4135a68a27f99a..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query68.sql +++ /dev/null @@ -1,44 +0,0 @@ -explain shape plan - - - - -select c_last_name - ,c_first_name - ,ca_city - ,bought_city - ,ss_ticket_number - ,extended_price - ,extended_tax - ,list_price - from (select ss_ticket_number - ,ss_customer_sk - ,ca_city bought_city - ,sum(ss_ext_sales_price) extended_price - ,sum(ss_ext_list_price) list_price - ,sum(ss_ext_tax) extended_tax - from store_sales - ,date_dim - ,store - ,household_demographics - ,customer_address - where store_sales.ss_sold_date_sk = date_dim.d_date_sk - and store_sales.ss_store_sk = store.s_store_sk - and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - and store_sales.ss_addr_sk = customer_address.ca_address_sk - and date_dim.d_dom between 1 and 2 - and (household_demographics.hd_dep_count = 8 or - household_demographics.hd_vehicle_count= -1) - and date_dim.d_year in (1998,1998+1,1998+2) - and store.s_city in ('Pleasant Hill','Five Points') - group by ss_ticket_number - ,ss_customer_sk - ,ss_addr_sk,ca_city) dn - ,customer - ,customer_address current_addr - where ss_customer_sk = c_customer_sk - and customer.c_current_addr_sk = current_addr.ca_address_sk - and current_addr.ca_city <> bought_city - order by c_last_name - ,ss_ticket_number - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query69.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query69.sql deleted file mode 100644 index 1d697aac8931486..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query69.sql +++ /dev/null @@ -1,49 +0,0 @@ -explain shape plan - - - - -select - cd_gender, - cd_marital_status, - cd_education_status, - count(*) cnt1, - cd_purchase_estimate, - count(*) cnt2, - cd_credit_rating, - count(*) cnt3 - from - customer c,customer_address ca,customer_demographics - where - c.c_current_addr_sk = ca.ca_address_sk and - ca_state in ('TX','VA','MI') and - cd_demo_sk = c.c_current_cdemo_sk and - exists (select * - from store_sales,date_dim - where c.c_customer_sk = ss_customer_sk and - ss_sold_date_sk = d_date_sk and - d_year = 2000 and - d_moy between 1 and 1+2) and - (not exists (select * - from web_sales,date_dim - where c.c_customer_sk = ws_bill_customer_sk and - ws_sold_date_sk = d_date_sk and - d_year = 2000 and - d_moy between 1 and 1+2) and - not exists (select * - from catalog_sales,date_dim - where c.c_customer_sk = cs_ship_customer_sk and - cs_sold_date_sk = d_date_sk and - d_year = 2000 and - d_moy between 1 and 1+2)) - group by cd_gender, - cd_marital_status, - cd_education_status, - cd_purchase_estimate, - cd_credit_rating - order by cd_gender, - cd_marital_status, - cd_education_status, - cd_purchase_estimate, - cd_credit_rating - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query7.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query7.sql deleted file mode 100644 index 6dc2aef3af10c76..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query7.sql +++ /dev/null @@ -1,23 +0,0 @@ -explain shape plan - - - - -select i_item_id, - avg(ss_quantity) agg1, - avg(ss_list_price) agg2, - avg(ss_coupon_amt) agg3, - avg(ss_sales_price) agg4 - from store_sales, customer_demographics, date_dim, item, promotion - where ss_sold_date_sk = d_date_sk and - ss_item_sk = i_item_sk and - ss_cdemo_sk = cd_demo_sk and - ss_promo_sk = p_promo_sk and - cd_gender = 'F' and - cd_marital_status = 'W' and - cd_education_status = 'College' and - (p_channel_email = 'N' or p_channel_event = 'N') and - d_year = 2001 - group by i_item_id - order by i_item_id - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query70.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query70.sql deleted file mode 100644 index 16f11e1c5d2cc84..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query70.sql +++ /dev/null @@ -1,40 +0,0 @@ -explain shape plan - - - - -select - sum(ss_net_profit) as total_sum - ,s_state - ,s_county - ,grouping(s_state)+grouping(s_county) as lochierarchy - ,rank() over ( - partition by grouping(s_state)+grouping(s_county), - case when grouping(s_county) = 0 then s_state end - order by sum(ss_net_profit) desc) as rank_within_parent - from - store_sales - ,date_dim d1 - ,store - where - d1.d_month_seq between 1213 and 1213+11 - and d1.d_date_sk = ss_sold_date_sk - and s_store_sk = ss_store_sk - and s_state in - ( select s_state - from (select s_state as s_state, - rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking - from store_sales, store, date_dim - where d_month_seq between 1213 and 1213+11 - and d_date_sk = ss_sold_date_sk - and s_store_sk = ss_store_sk - group by s_state - ) tmp1 - where ranking <= 5 - ) - group by rollup(s_state,s_county) - order by - lochierarchy desc - ,case when lochierarchy = 0 then s_state end - ,rank_within_parent - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query71.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query71.sql deleted file mode 100644 index d7f3fb56c6f1dcb..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query71.sql +++ /dev/null @@ -1,42 +0,0 @@ -explain shape plan - - - - -select i_brand_id brand_id, i_brand brand,t_hour,t_minute, - sum(ext_price) ext_price - from item, (select ws_ext_sales_price as ext_price, - ws_sold_date_sk as sold_date_sk, - ws_item_sk as sold_item_sk, - ws_sold_time_sk as time_sk - from web_sales,date_dim - where d_date_sk = ws_sold_date_sk - and d_moy=12 - and d_year=1998 - union all - select cs_ext_sales_price as ext_price, - cs_sold_date_sk as sold_date_sk, - cs_item_sk as sold_item_sk, - cs_sold_time_sk as time_sk - from catalog_sales,date_dim - where d_date_sk = cs_sold_date_sk - and d_moy=12 - and d_year=1998 - union all - select ss_ext_sales_price as ext_price, - ss_sold_date_sk as sold_date_sk, - ss_item_sk as sold_item_sk, - ss_sold_time_sk as time_sk - from store_sales,date_dim - where d_date_sk = ss_sold_date_sk - and d_moy=12 - and d_year=1998 - ) tmp,time_dim - where - sold_item_sk = i_item_sk - and i_manager_id=1 - and time_sk = t_time_sk - and (t_meal_time = 'breakfast' or t_meal_time = 'dinner') - group by i_brand, i_brand_id,t_hour,t_minute - order by ext_price desc, i_brand_id - ; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query72.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query72.sql deleted file mode 100644 index d6a31fe89af56af..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query72.sql +++ /dev/null @@ -1,31 +0,0 @@ -explain shape plan - - - - -select i_item_desc - ,w_warehouse_name - ,d1.d_week_seq - ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo - ,sum(case when p_promo_sk is not null then 1 else 0 end) promo - ,count(*) total_cnt -from catalog_sales -join inventory on (cs_item_sk = inv_item_sk) -join warehouse on (w_warehouse_sk=inv_warehouse_sk) -join item on (i_item_sk = cs_item_sk) -join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) -join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) -join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) -join date_dim d2 on (inv_date_sk = d2.d_date_sk) -join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) -left outer join promotion on (cs_promo_sk=p_promo_sk) -left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) -where d1.d_week_seq = d2.d_week_seq - and inv_quantity_on_hand < cs_quantity - and d3.d_date > d1.d_date + 5 - and hd_buy_potential = '501-1000' - and d1.d_year = 2002 - and cd_marital_status = 'W' -group by i_item_desc,w_warehouse_name,d1.d_week_seq -order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query73.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query73.sql deleted file mode 100644 index 256711919de4cc1..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query73.sql +++ /dev/null @@ -1,30 +0,0 @@ -explain shape plan - - - - -select c_last_name - ,c_first_name - ,c_salutation - ,c_preferred_cust_flag - ,ss_ticket_number - ,cnt from - (select ss_ticket_number - ,ss_customer_sk - ,count(*) cnt - from store_sales,date_dim,store,household_demographics - where store_sales.ss_sold_date_sk = date_dim.d_date_sk - and store_sales.ss_store_sk = store.s_store_sk - and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - and date_dim.d_dom between 1 and 2 - and (household_demographics.hd_buy_potential = '501-1000' or - household_demographics.hd_buy_potential = 'Unknown') - and household_demographics.hd_vehicle_count > 0 - and case when household_demographics.hd_vehicle_count > 0 then - household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 - and date_dim.d_year in (2000,2000+1,2000+2) - and store.s_county in ('Fairfield County','Walker County','Daviess County','Barrow County') - group by ss_ticket_number,ss_customer_sk) dj,customer - where ss_customer_sk = c_customer_sk - and cnt between 1 and 5 - order by cnt desc, c_last_name asc; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query74.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query74.sql deleted file mode 100644 index 966defd6f720ce3..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query74.sql +++ /dev/null @@ -1,62 +0,0 @@ -explain shape plan - - - -with year_total as ( - select c_customer_id customer_id - ,c_first_name customer_first_name - ,c_last_name customer_last_name - ,d_year as year - ,stddev_samp(ss_net_paid) year_total - ,'s' sale_type - from customer - ,store_sales - ,date_dim - where c_customer_sk = ss_customer_sk - and ss_sold_date_sk = d_date_sk - and d_year in (1999,1999+1) - group by c_customer_id - ,c_first_name - ,c_last_name - ,d_year - union all - select c_customer_id customer_id - ,c_first_name customer_first_name - ,c_last_name customer_last_name - ,d_year as year - ,stddev_samp(ws_net_paid) year_total - ,'w' sale_type - from customer - ,web_sales - ,date_dim - where c_customer_sk = ws_bill_customer_sk - and ws_sold_date_sk = d_date_sk - and d_year in (1999,1999+1) - group by c_customer_id - ,c_first_name - ,c_last_name - ,d_year - ) - select - t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name - from year_total t_s_firstyear - ,year_total t_s_secyear - ,year_total t_w_firstyear - ,year_total t_w_secyear - where t_s_secyear.customer_id = t_s_firstyear.customer_id - and t_s_firstyear.customer_id = t_w_secyear.customer_id - and t_s_firstyear.customer_id = t_w_firstyear.customer_id - and t_s_firstyear.sale_type = 's' - and t_w_firstyear.sale_type = 'w' - and t_s_secyear.sale_type = 's' - and t_w_secyear.sale_type = 'w' - and t_s_firstyear.year = 1999 - and t_s_secyear.year = 1999+1 - and t_w_firstyear.year = 1999 - and t_w_secyear.year = 1999+1 - and t_s_firstyear.year_total > 0 - and t_w_firstyear.year_total > 0 - and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end - > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end - order by 2,1,3 -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query76.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query76.sql deleted file mode 100644 index 1499d7708c85693..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query76.sql +++ /dev/null @@ -1,26 +0,0 @@ -explain shape plan - - - - -select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( - SELECT 'store' as channel, 'ss_hdemo_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price - FROM store_sales, item, date_dim - WHERE ss_hdemo_sk IS NULL - AND ss_sold_date_sk=d_date_sk - AND ss_item_sk=i_item_sk - UNION ALL - SELECT 'web' as channel, 'ws_bill_addr_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price - FROM web_sales, item, date_dim - WHERE ws_bill_addr_sk IS NULL - AND ws_sold_date_sk=d_date_sk - AND ws_item_sk=i_item_sk - UNION ALL - SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price - FROM catalog_sales, item, date_dim - WHERE cs_warehouse_sk IS NULL - AND cs_sold_date_sk=d_date_sk - AND cs_item_sk=i_item_sk) foo -GROUP BY channel, col_name, d_year, d_qoy, i_category -ORDER BY channel, col_name, d_year, d_qoy, i_category -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query78.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query78.sql deleted file mode 100644 index 75945184ab0b1f8..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query78.sql +++ /dev/null @@ -1,60 +0,0 @@ -explain shape plan - - - - -with ws as - (select d_year AS ws_sold_year, ws_item_sk, - ws_bill_customer_sk ws_customer_sk, - sum(ws_quantity) ws_qty, - sum(ws_wholesale_cost) ws_wc, - sum(ws_sales_price) ws_sp - from web_sales - left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk - join date_dim on ws_sold_date_sk = d_date_sk - where wr_order_number is null - group by d_year, ws_item_sk, ws_bill_customer_sk - ), -cs as - (select d_year AS cs_sold_year, cs_item_sk, - cs_bill_customer_sk cs_customer_sk, - sum(cs_quantity) cs_qty, - sum(cs_wholesale_cost) cs_wc, - sum(cs_sales_price) cs_sp - from catalog_sales - left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk - join date_dim on cs_sold_date_sk = d_date_sk - where cr_order_number is null - group by d_year, cs_item_sk, cs_bill_customer_sk - ), -ss as - (select d_year AS ss_sold_year, ss_item_sk, - ss_customer_sk, - sum(ss_quantity) ss_qty, - sum(ss_wholesale_cost) ss_wc, - sum(ss_sales_price) ss_sp - from store_sales - left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk - join date_dim on ss_sold_date_sk = d_date_sk - where sr_ticket_number is null - group by d_year, ss_item_sk, ss_customer_sk - ) - select -ss_item_sk, -round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio, -ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price, -coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty, -coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost, -coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price -from ss -left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk) -left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk) -where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=2000 -order by - ss_item_sk, - ss_qty desc, ss_wc desc, ss_sp desc, - other_chan_qty, - other_chan_wholesale_cost, - other_chan_sales_price, - ratio -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query79.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query79.sql deleted file mode 100644 index 6a4e8560200ee02..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query79.sql +++ /dev/null @@ -1,25 +0,0 @@ -explain shape plan - - - - -select - c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit - from - (select ss_ticket_number - ,ss_customer_sk - ,store.s_city - ,sum(ss_coupon_amt) amt - ,sum(ss_net_profit) profit - from store_sales,date_dim,store,household_demographics - where store_sales.ss_sold_date_sk = date_dim.d_date_sk - and store_sales.ss_store_sk = store.s_store_sk - and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk - and (household_demographics.hd_dep_count = 5 or household_demographics.hd_vehicle_count > 4) - and date_dim.d_dow = 1 - and date_dim.d_year in (1998,1998+1,1998+2) - and store.s_number_employees between 200 and 295 - group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer - where ss_customer_sk = c_customer_sk - order by c_last_name,c_first_name,substr(s_city,1,30), profit -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query81.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query81.sql deleted file mode 100644 index 7e67ca00a231b1d..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query81.sql +++ /dev/null @@ -1,33 +0,0 @@ -explain shape plan - - - - -with customer_total_return as - (select cr_returning_customer_sk as ctr_customer_sk - ,ca_state as ctr_state, - sum(cr_return_amt_inc_tax) as ctr_total_return - from catalog_returns - ,date_dim - ,customer_address - where cr_returned_date_sk = d_date_sk - and d_year =2002 - and cr_returning_addr_sk = ca_address_sk - group by cr_returning_customer_sk - ,ca_state ) - select c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name - ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset - ,ca_location_type,ctr_total_return - from customer_total_return ctr1 - ,customer_address - ,customer - where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 - from customer_total_return ctr2 - where ctr1.ctr_state = ctr2.ctr_state) - and ca_address_sk = c_current_addr_sk - and ca_state = 'CA' - and ctr1.ctr_customer_sk = c_customer_sk - order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name - ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset - ,ca_location_type,ctr_total_return - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query82.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query82.sql deleted file mode 100644 index 6ec7b15ffca9cc5..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query82.sql +++ /dev/null @@ -1,19 +0,0 @@ -explain shape plan - - - - -select i_item_id - ,i_item_desc - ,i_current_price - from item, inventory, date_dim, store_sales - where i_current_price between 17 and 17+30 - and inv_item_sk = i_item_sk - and d_date_sk=inv_date_sk - and d_date between cast('1999-07-09' as date) and (cast('1999-07-09' as date) + interval 60 day) - and i_manufact_id in (639,169,138,339) - and inv_quantity_on_hand between 100 and 500 - and ss_item_sk = i_item_sk - group by i_item_id,i_item_desc,i_current_price - order by i_item_id - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query83.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query83.sql deleted file mode 100644 index a35792debce6d03..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query83.sql +++ /dev/null @@ -1,69 +0,0 @@ -explain shape plan - - - - -with sr_items as - (select i_item_id item_id, - sum(sr_return_quantity) sr_item_qty - from store_returns, - item, - date_dim - where sr_item_sk = i_item_sk - and d_date in - (select d_date - from date_dim - where d_week_seq in - (select d_week_seq - from date_dim - where d_date in ('2001-06-06','2001-09-02','2001-11-11'))) - and sr_returned_date_sk = d_date_sk - group by i_item_id), - cr_items as - (select i_item_id item_id, - sum(cr_return_quantity) cr_item_qty - from catalog_returns, - item, - date_dim - where cr_item_sk = i_item_sk - and d_date in - (select d_date - from date_dim - where d_week_seq in - (select d_week_seq - from date_dim - where d_date in ('2001-06-06','2001-09-02','2001-11-11'))) - and cr_returned_date_sk = d_date_sk - group by i_item_id), - wr_items as - (select i_item_id item_id, - sum(wr_return_quantity) wr_item_qty - from web_returns, - item, - date_dim - where wr_item_sk = i_item_sk - and d_date in - (select d_date - from date_dim - where d_week_seq in - (select d_week_seq - from date_dim - where d_date in ('2001-06-06','2001-09-02','2001-11-11'))) - and wr_returned_date_sk = d_date_sk - group by i_item_id) - select sr_items.item_id - ,sr_item_qty - ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev - ,cr_item_qty - ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev - ,wr_item_qty - ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev - ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average - from sr_items - ,cr_items - ,wr_items - where sr_items.item_id=cr_items.item_id - and sr_items.item_id=wr_items.item_id - order by sr_items.item_id - ,sr_item_qty - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query84.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query84.sql deleted file mode 100644 index 9322faf1403c9cc..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query84.sql +++ /dev/null @@ -1,23 +0,0 @@ -explain shape plan - - - - -select c_customer_id as customer_id - , concat(concat(coalesce(c_last_name,''), ','), coalesce(c_first_name,'')) as customername - from customer - ,customer_address - ,customer_demographics - ,household_demographics - ,income_band - ,store_returns - where ca_city = 'Oakwood' - and c_current_addr_sk = ca_address_sk - and ib_lower_bound >= 5806 - and ib_upper_bound <= 5806 + 50000 - and ib_income_band_sk = hd_income_band_sk - and cd_demo_sk = c_current_cdemo_sk - and hd_demo_sk = c_current_hdemo_sk - and sr_cdemo_sk = cd_demo_sk - order by c_customer_id - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query85.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query85.sql deleted file mode 100644 index 2c8a0eb1fd8ec21..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query85.sql +++ /dev/null @@ -1,86 +0,0 @@ -explain shape plan - - - - -select substr(r_reason_desc,1,20) - ,avg(ws_quantity) - ,avg(wr_refunded_cash) - ,avg(wr_fee) - from web_sales, web_returns, web_page, customer_demographics cd1, - customer_demographics cd2, customer_address, date_dim, reason - where ws_web_page_sk = wp_web_page_sk - and ws_item_sk = wr_item_sk - and ws_order_number = wr_order_number - and ws_sold_date_sk = d_date_sk and d_year = 2000 - and cd1.cd_demo_sk = wr_refunded_cdemo_sk - and cd2.cd_demo_sk = wr_returning_cdemo_sk - and ca_address_sk = wr_refunded_addr_sk - and r_reason_sk = wr_reason_sk - and - ( - ( - cd1.cd_marital_status = 'M' - and - cd1.cd_marital_status = cd2.cd_marital_status - and - cd1.cd_education_status = '4 yr Degree' - and - cd1.cd_education_status = cd2.cd_education_status - and - ws_sales_price between 100.00 and 150.00 - ) - or - ( - cd1.cd_marital_status = 'S' - and - cd1.cd_marital_status = cd2.cd_marital_status - and - cd1.cd_education_status = 'Secondary' - and - cd1.cd_education_status = cd2.cd_education_status - and - ws_sales_price between 50.00 and 100.00 - ) - or - ( - cd1.cd_marital_status = 'W' - and - cd1.cd_marital_status = cd2.cd_marital_status - and - cd1.cd_education_status = 'Advanced Degree' - and - cd1.cd_education_status = cd2.cd_education_status - and - ws_sales_price between 150.00 and 200.00 - ) - ) - and - ( - ( - ca_country = 'United States' - and - ca_state in ('FL', 'TX', 'DE') - and ws_net_profit between 100 and 200 - ) - or - ( - ca_country = 'United States' - and - ca_state in ('IN', 'ND', 'ID') - and ws_net_profit between 150 and 300 - ) - or - ( - ca_country = 'United States' - and - ca_state in ('MT', 'IL', 'OH') - and ws_net_profit between 50 and 250 - ) - ) -group by r_reason_desc -order by substr(r_reason_desc,1,20) - ,avg(ws_quantity) - ,avg(wr_refunded_cash) - ,avg(wr_fee) -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query86.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query86.sql deleted file mode 100644 index 7a1e0b6811a0ba8..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query86.sql +++ /dev/null @@ -1,28 +0,0 @@ -explain shape plan - - - - -select - sum(ws_net_paid) as total_sum - ,i_category - ,i_class - ,grouping(i_category)+grouping(i_class) as lochierarchy - ,rank() over ( - partition by grouping(i_category)+grouping(i_class), - case when grouping(i_class) = 0 then i_category end - order by sum(ws_net_paid) desc) as rank_within_parent - from - web_sales - ,date_dim d1 - ,item - where - d1.d_month_seq between 1224 and 1224+11 - and d1.d_date_sk = ws_sold_date_sk - and i_item_sk = ws_item_sk - group by rollup(i_category,i_class) - order by - lochierarchy desc, - case when lochierarchy = 0 then i_category end, - rank_within_parent - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query87.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query87.sql deleted file mode 100644 index 12d33bd09cffa2d..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query87.sql +++ /dev/null @@ -1,25 +0,0 @@ -explain shape plan - - - - -select count(*) -from ((select distinct c_last_name, c_first_name, d_date - from store_sales, date_dim, customer - where store_sales.ss_sold_date_sk = date_dim.d_date_sk - and store_sales.ss_customer_sk = customer.c_customer_sk - and d_month_seq between 1184 and 1184+11) - except - (select distinct c_last_name, c_first_name, d_date - from catalog_sales, date_dim, customer - where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk - and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk - and d_month_seq between 1184 and 1184+11) - except - (select distinct c_last_name, c_first_name, d_date - from web_sales, date_dim, customer - where web_sales.ws_sold_date_sk = date_dim.d_date_sk - and web_sales.ws_bill_customer_sk = customer.c_customer_sk - and d_month_seq between 1184 and 1184+11) -) cool_cust -; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query89.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query89.sql deleted file mode 100644 index 527dcd39265bd83..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query89.sql +++ /dev/null @@ -1,30 +0,0 @@ -explain shape plan - - - - -select * -from( -select i_category, i_class, i_brand, - s_store_name, s_company_name, - d_moy, - sum(ss_sales_price) sum_sales, - avg(sum(ss_sales_price)) over - (partition by i_category, i_brand, s_store_name, s_company_name) - avg_monthly_sales -from item, store_sales, date_dim, store -where ss_item_sk = i_item_sk and - ss_sold_date_sk = d_date_sk and - ss_store_sk = s_store_sk and - d_year in (1999) and - ((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') - )) -group by i_category, i_class, i_brand, - s_store_name, s_company_name, d_moy) tmp1 -where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 -order by sum_sales - avg_monthly_sales, s_store_name -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query9.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query9.sql deleted file mode 100644 index 4b269ee719d66c5..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query9.sql +++ /dev/null @@ -1,53 +0,0 @@ -explain shape plan - - - - -select case when (select count(*) - from store_sales - where ss_quantity between 1 and 20) > 2972190 - then (select avg(ss_ext_sales_price) - from store_sales - where ss_quantity between 1 and 20) - else (select avg(ss_net_profit) - from store_sales - where ss_quantity between 1 and 20) end bucket1 , - case when (select count(*) - from store_sales - where ss_quantity between 21 and 40) > 4505785 - then (select avg(ss_ext_sales_price) - from store_sales - where ss_quantity between 21 and 40) - else (select avg(ss_net_profit) - from store_sales - where ss_quantity between 21 and 40) end bucket2, - case when (select count(*) - from store_sales - where ss_quantity between 41 and 60) > 1575726 - then (select avg(ss_ext_sales_price) - from store_sales - where ss_quantity between 41 and 60) - else (select avg(ss_net_profit) - from store_sales - where ss_quantity between 41 and 60) end bucket3, - case when (select count(*) - from store_sales - where ss_quantity between 61 and 80) > 3188917 - then (select avg(ss_ext_sales_price) - from store_sales - where ss_quantity between 61 and 80) - else (select avg(ss_net_profit) - from store_sales - where ss_quantity between 61 and 80) end bucket4, - case when (select count(*) - from store_sales - where ss_quantity between 81 and 100) > 3525216 - then (select avg(ss_ext_sales_price) - from store_sales - where ss_quantity between 81 and 100) - else (select avg(ss_net_profit) - from store_sales - where ss_quantity between 81 and 100) end bucket5 -from reason -where r_reason_sk = 1 -; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query90.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query90.sql deleted file mode 100644 index b1531f5069dad21..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query90.sql +++ /dev/null @@ -1,24 +0,0 @@ -explain shape plan - - - - -select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio - from ( select count(*) amc - from web_sales, household_demographics , time_dim, web_page - where ws_sold_time_sk = time_dim.t_time_sk - and ws_ship_hdemo_sk = household_demographics.hd_demo_sk - and ws_web_page_sk = web_page.wp_web_page_sk - and time_dim.t_hour between 10 and 10+1 - and household_demographics.hd_dep_count = 2 - and web_page.wp_char_count between 5000 and 5200) at, - ( select count(*) pmc - from web_sales, household_demographics , time_dim, web_page - where ws_sold_time_sk = time_dim.t_time_sk - and ws_ship_hdemo_sk = household_demographics.hd_demo_sk - and ws_web_page_sk = web_page.wp_web_page_sk - and time_dim.t_hour between 16 and 16+1 - and household_demographics.hd_dep_count = 2 - and web_page.wp_char_count between 5000 and 5200) pt - order by am_pm_ratio - limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query91.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query91.sql deleted file mode 100644 index 371770012987092..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query91.sql +++ /dev/null @@ -1,33 +0,0 @@ -explain shape plan - - - - -select - cc_call_center_id Call_Center, - cc_name Call_Center_Name, - cc_manager Manager, - sum(cr_net_loss) Returns_Loss -from - call_center, - catalog_returns, - date_dim, - customer, - customer_address, - customer_demographics, - household_demographics -where - cr_call_center_sk = cc_call_center_sk -and cr_returned_date_sk = d_date_sk -and cr_returning_customer_sk= c_customer_sk -and cd_demo_sk = c_current_cdemo_sk -and hd_demo_sk = c_current_hdemo_sk -and ca_address_sk = c_current_addr_sk -and d_year = 2001 -and d_moy = 11 -and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') - or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) -and hd_buy_potential like '1001-5000%' -and ca_gmt_offset = -6 -group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status -order by sum(cr_net_loss) desc; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query92.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query92.sql deleted file mode 100644 index 8bcacc0f3360cec..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query92.sql +++ /dev/null @@ -1,32 +0,0 @@ -explain shape plan - - - - -select - sum(ws_ext_discount_amt) as "Excess Discount Amount" -from - web_sales - ,item - ,date_dim -where -i_manufact_id = 320 -and i_item_sk = ws_item_sk -and d_date between '2002-02-26' and - (cast('2002-02-26' as date) + interval 90 day) -and d_date_sk = ws_sold_date_sk -and ws_ext_discount_amt - > ( - SELECT - 1.3 * avg(ws_ext_discount_amt) - FROM - web_sales - ,date_dim - WHERE - ws_item_sk = i_item_sk - and d_date between '2002-02-26' and - (cast('2002-02-26' as date) + interval 90 day) - and d_date_sk = ws_sold_date_sk - ) -order by sum(ws_ext_discount_amt) -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query93.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query93.sql deleted file mode 100644 index 66a71169525b61f..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query93.sql +++ /dev/null @@ -1,20 +0,0 @@ -explain shape plan - - - - -select ss_customer_sk - ,sum(act_sales) sumsales - from (select ss_item_sk - ,ss_ticket_number - ,ss_customer_sk - ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price - else (ss_quantity*ss_sales_price) end act_sales - from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk - and sr_ticket_number = ss_ticket_number) - ,reason - where sr_reason_sk = r_reason_sk - and r_reason_desc = 'duplicate purchase') t - group by ss_customer_sk - order by sumsales, ss_customer_sk -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query94.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query94.sql deleted file mode 100644 index e47ea859d440d2f..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query94.sql +++ /dev/null @@ -1,31 +0,0 @@ -explain shape plan - - - - -select - count(distinct ws_order_number) as "order count" - ,sum(ws_ext_ship_cost) as "total shipping cost" - ,sum(ws_net_profit) as "total net profit" -from - web_sales ws1 - ,date_dim - ,customer_address - ,web_site -where - d_date between '2000-2-01' and - (cast('2000-2-01' as date) + interval 60 day) -and ws1.ws_ship_date_sk = d_date_sk -and ws1.ws_ship_addr_sk = ca_address_sk -and ca_state = 'OK' -and ws1.ws_web_site_sk = web_site_sk -and web_company_name = 'pri' -and exists (select * - from web_sales ws2 - where ws1.ws_order_number = ws2.ws_order_number - and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) -and not exists(select * - from web_returns wr1 - where ws1.ws_order_number = wr1.wr_order_number) -order by count(distinct ws_order_number) -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query95.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query95.sql deleted file mode 100644 index 3b23dc4a2d653fa..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query95.sql +++ /dev/null @@ -1,32 +0,0 @@ -explain shape plan - - -with ws_wh as -(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2 - from web_sales ws1,web_sales ws2 - where ws1.ws_order_number = ws2.ws_order_number - and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) - select - count(distinct ws_order_number) as "order count" - ,sum(ws_ext_ship_cost) as "total shipping cost" - ,sum(ws_net_profit) as "total net profit" -from - web_sales ws1 - ,date_dim - ,customer_address - ,web_site -where - d_date between '1999-2-01' and - (cast('1999-2-01' as date) + interval 60 day) -and ws1.ws_ship_date_sk = d_date_sk -and ws1.ws_ship_addr_sk = ca_address_sk -and ca_state = 'NC' -and ws1.ws_web_site_sk = web_site_sk -and web_company_name = 'pri' -and ws1.ws_order_number in (select ws_order_number - from ws_wh) -and ws1.ws_order_number in (select wr_order_number - from web_returns,ws_wh - where wr_order_number = ws_wh.ws_order_number) -order by count(distinct ws_order_number) -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query96.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query96.sql deleted file mode 100644 index e55d6679f3fdfae..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query96.sql +++ /dev/null @@ -1,18 +0,0 @@ -explain shape plan - - - - -select count(*) -from store_sales - ,household_demographics - ,time_dim, store -where ss_sold_time_sk = time_dim.t_time_sk - and ss_hdemo_sk = household_demographics.hd_demo_sk - and ss_store_sk = s_store_sk - and time_dim.t_hour = 8 - and time_dim.t_minute >= 30 - and household_demographics.hd_dep_count = 3 - and store.s_store_name = 'ese' -order by count(*) -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query97.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query97.sql deleted file mode 100644 index 06ba78ca27bbcf0..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query97.sql +++ /dev/null @@ -1,27 +0,0 @@ -explain shape plan - - - - -with ssci as ( -select ss_customer_sk customer_sk - ,ss_item_sk item_sk -from store_sales,date_dim -where ss_sold_date_sk = d_date_sk - and d_month_seq between 1214 and 1214 + 11 -group by ss_customer_sk - ,ss_item_sk), -csci as( - select cs_bill_customer_sk customer_sk - ,cs_item_sk item_sk -from catalog_sales,date_dim -where cs_sold_date_sk = d_date_sk - and d_month_seq between 1214 and 1214 + 11 -group by cs_bill_customer_sk - ,cs_item_sk) - select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only - ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only - ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog -from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk - and ssci.item_sk = csci.item_sk) -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query98.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query98.sql deleted file mode 100644 index 7fb3b1a2b8f4a63..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query98.sql +++ /dev/null @@ -1,35 +0,0 @@ -explain shape plan - - - - -select i_item_id - ,i_item_desc - ,i_category - ,i_class - ,i_current_price - ,sum(ss_ext_sales_price) as itemrevenue - ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over - (partition by i_class) as revenueratio -from - store_sales - ,item - ,date_dim -where - ss_item_sk = i_item_sk - and i_category in ('Sports', 'Music', 'Shoes') - and ss_sold_date_sk = d_date_sk - and d_date between cast('2002-05-20' as date) - and (cast('2002-05-20' as date) + interval 30 day) -group by - i_item_id - ,i_item_desc - ,i_category - ,i_class - ,i_current_price -order by - i_category - ,i_class - ,i_item_id - ,i_item_desc - ,revenueratio; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query99.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query99.sql deleted file mode 100644 index d221dba8fb5921d..000000000000000 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query99.sql +++ /dev/null @@ -1,37 +0,0 @@ -explain shape plan - - - - -select - substr(w_warehouse_name,1,20) - ,sm_type - ,cc_name - ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" - ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and - (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" - ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and - (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" - ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and - (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" - ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as ">120 days" -from - catalog_sales - ,warehouse - ,ship_mode - ,call_center - ,date_dim -where - d_month_seq between 1224 and 1224 + 11 -and cs_ship_date_sk = d_date_sk -and cs_warehouse_sk = w_warehouse_sk -and cs_ship_mode_sk = sm_ship_mode_sk -and cs_call_center_sk = cc_call_center_sk -group by - substr(w_warehouse_name,1,20) - ,sm_type - ,cc_name -order by substr(w_warehouse_name,1,20) - ,sm_type - ,cc_name -limit 100; diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf.tmpl b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf.tmpl new file mode 100644 index 000000000000000..068d9acb39298cd --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf.tmpl @@ -0,0 +1,57 @@ +/* + * 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("ds_rf{--}") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + {query} + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.{--}" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("{rfs}", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/case.tmpl b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/shape.tmpl similarity index 95% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/case.tmpl rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/shape.tmpl index 8d930d4f86d422e..8f1d4bf10486f33 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/case.tmpl +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/shape.tmpl @@ -31,6 +31,7 @@ suite("query{--}") { sql 'set enable_nereids_timeout = false' qt_ds_shape_{--} ''' + explain shape plan {query} ''' } diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf1.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf1.groovy new file mode 100644 index 000000000000000..9bb1877d1f27490 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf1.groovy @@ -0,0 +1,79 @@ +/* + * 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("ds_rf1") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with customer_total_return as +(select sr_customer_sk as ctr_customer_sk +,sr_store_sk as ctr_store_sk +,sum(SR_FEE) as ctr_total_return +from store_returns +,date_dim +where sr_returned_date_sk = d_date_sk +and d_year =2000 +group by sr_customer_sk +,sr_store_sk) + select c_customer_id +from customer_total_return ctr1 +,store +,customer +where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 +from customer_total_return ctr2 +where ctr1.ctr_store_sk = ctr2.ctr_store_sk) +and s_store_sk = ctr1.ctr_store_sk +and s_state = 'SD' +and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.1" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF0[d_date_sk->[sr_returned_date_sk],RF2[ctr_customer_sk->[c_customer_sk],RF1[s_store_sk->[ctr_store_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf10.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf10.groovy new file mode 100644 index 000000000000000..9bc4fd4c70cd664 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf10.groovy @@ -0,0 +1,113 @@ +/* + * 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("ds_rf10") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_county in ('Storey County','Marquette County','Warren County','Cochran County','Kandiyohi County') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2001 and + d_moy between 1 and 1+3) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2001 and + d_moy between 1 ANd 1+3) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2001 and + d_moy between 1 and 1+3)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.10" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[c_current_cdemo_sk->[cd_demo_sk],RF4[c_customer_sk->[ss_customer_sk],RF3[d_date_sk->[ss_sold_date_sk],RF2[ca_address_sk->[c_current_addr_sk],RF1[d_date_sk->[ws_sold_date_sk],RF0[d_date_sk->[cs_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query11.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf11.groovy similarity index 55% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query11.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf11.groovy index 132a82d4f8dc260..c36b43a5ac413dc 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query11.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf11.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -with year_total as ( +/* + * 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("ds_rf11") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with year_total as ( select c_customer_id customer_id ,c_first_name customer_first_name ,c_last_name customer_last_name @@ -81,3 +110,26 @@ with year_total as ( ,t_s_secyear.customer_last_name ,t_s_secyear.customer_birth_country limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.11" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[c_customer_sk->[ss_customer_sk],RF0[d_date_sk->[ss_sold_date_sk],RF3[c_customer_sk->[ws_bill_customer_sk],RF2[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf12.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf12.groovy new file mode 100644 index 000000000000000..35d5701519dc0ce --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf12.groovy @@ -0,0 +1,88 @@ +/* + * 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("ds_rf12") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ws_ext_sales_price) as itemrevenue + ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over + (partition by i_class) as revenueratio +from + web_sales + ,item + ,date_dim +where + ws_item_sk = i_item_sk + and i_category in ('Books', 'Sports', 'Men') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1998-04-06' as date) + and (cast('1998-04-06' as date) + interval 30 day) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.12" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[i_item_sk->[ws_item_sk],RF0[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf13.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf13.groovy new file mode 100644 index 000000000000000..2961139fa0d4948 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf13.groovy @@ -0,0 +1,106 @@ +/* + * 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("ds_rf13") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select avg(ss_quantity) + ,avg(ss_ext_sales_price) + ,avg(ss_ext_wholesale_cost) + ,sum(ss_ext_wholesale_cost) + from store_sales + ,store + ,customer_demographics + ,household_demographics + ,customer_address + ,date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and((ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'D' + and cd_education_status = 'Unknown' + and ss_sales_price between 100.00 and 150.00 + and hd_dep_count = 3 + )or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'S' + and cd_education_status = 'College' + and ss_sales_price between 50.00 and 100.00 + and hd_dep_count = 1 + ) or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'M' + and cd_education_status = '4 yr Degree' + and ss_sales_price between 150.00 and 200.00 + and hd_dep_count = 1 + )) + and((ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('SD', 'KS', 'MI') + and ss_net_profit between 100 and 200 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('MO', 'ND', 'CO') + and ss_net_profit between 150 and 300 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('NH', 'OH', 'TX') + and ss_net_profit between 50 and 250 + )) +; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.13" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF4[ss_store_sk->[s_store_sk],RF3[ss_sold_date_sk->[d_date_sk],RF2[ss_cdemo_sk->[cd_demo_sk],RF1[ss_hdemo_sk->[hd_demo_sk],RF0[ca_address_sk->[ss_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query14.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf14.groovy similarity index 55% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query14.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf14.groovy index f74c0d1ab0d8841..4f88f7b762c2ea5 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query14.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf14.groovy @@ -1,7 +1,38 @@ -explain shape plan - - -with cross_items as +/* + * 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("ds_rf14") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with cross_items as (select i_item_sk ss_item_sk from item, (select iss.i_brand_id brand_id @@ -102,3 +133,26 @@ with cross_items as group by rollup (channel, i_brand_id,i_class_id,i_category_id) order by channel,i_brand_id,i_class_id,i_category_id limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.14" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF6[class_id->[i_class_id],RF7[category_id->[i_category_id],RF8[brand_id->[i_brand_id],RF1[i_item_sk->[ss_item_sk],RF0[d_date_sk->[ss_sold_date_sk],RF3[i_item_sk->[cs_item_sk],RF2[d_date_sk->[cs_sold_date_sk],RF5[i_item_sk->[ws_item_sk],RF4[d_date_sk->[ws_sold_date_sk],RF9[d_date_sk->[ss_sold_date_sk],RF10[d_date_sk->[cs_sold_date_sk],RF11[d_date_sk->[ws_sold_date_sk],RF14[ss_item_sk->[ss_item_sk],RF13[i_item_sk->[ss_item_sk],RF12[d_date_sk->[ss_sold_date_sk],RF17[cs_item_sk->[ss_item_sk],RF16[i_item_sk->[cs_item_sk],RF15[d_date_sk->[cs_sold_date_sk],RF20[ws_item_sk->[ss_item_sk],RF19[i_item_sk->[ws_item_sk],RF18[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf15.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf15.groovy new file mode 100644 index 000000000000000..94d432cef20f08a --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf15.groovy @@ -0,0 +1,74 @@ +/* + * 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("ds_rf15") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select ca_zip + ,sum(cs_sales_price) + from catalog_sales + ,customer + ,customer_address + ,date_dim + where cs_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', + '85392', '85460', '80348', '81792') + or ca_state in ('CA','WA','GA') + or cs_sales_price > 500) + and cs_sold_date_sk = d_date_sk + and d_qoy = 1 and d_year = 2001 + group by ca_zip + order by ca_zip + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.15" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[c_customer_sk->[cs_bill_customer_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[ca_address_sk->[c_current_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf16.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf16.groovy new file mode 100644 index 000000000000000..c0ac901f3d2ad6d --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf16.groovy @@ -0,0 +1,85 @@ +/* + * 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("ds_rf16") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + count(distinct cs_order_number) as "order count" + ,sum(cs_ext_ship_cost) as "total shipping cost" + ,sum(cs_net_profit) as "total net profit" +from + catalog_sales cs1 + ,date_dim + ,customer_address + ,call_center +where + d_date between '2002-4-01' and + (cast('2002-4-01' as date) + interval 60 day) +and cs1.cs_ship_date_sk = d_date_sk +and cs1.cs_ship_addr_sk = ca_address_sk +and ca_state = 'WV' +and cs1.cs_call_center_sk = cc_call_center_sk +and cc_county in ('Ziebach County','Luce County','Richland County','Daviess County', + 'Barrow County' +) +and exists (select * + from catalog_sales cs2 + where cs1.cs_order_number = cs2.cs_order_number + and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) +and not exists(select * + from catalog_returns cr1 + where cs1.cs_order_number = cr1.cr_order_number) +order by count(distinct cs_order_number) +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.16" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF4[d_date_sk->[cs_ship_date_sk],RF3[cs_order_number->[cs_order_number],RF2[cc_call_center_sk->[cs_call_center_sk],RF1[cs_order_number->[cr_order_number],RF0[ca_address_sk->[cs_ship_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf17.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf17.groovy new file mode 100644 index 000000000000000..548d6104b03a983 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf17.groovy @@ -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. + */ + +suite("ds_rf17") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id + ,i_item_desc + ,s_state + ,count(ss_quantity) as store_sales_quantitycount + ,avg(ss_quantity) as store_sales_quantityave + ,stddev_samp(ss_quantity) as store_sales_quantitystdev + ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov + ,count(sr_return_quantity) as store_returns_quantitycount + ,avg(sr_return_quantity) as store_returns_quantityave + ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev + ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov + ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave + ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev + ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov + from store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where d1.d_quarter_name = '2001Q1' + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_quarter_name in ('2001Q1','2001Q2','2001Q3') + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_quarter_name in ('2001Q1','2001Q2','2001Q3') + group by i_item_id + ,i_item_desc + ,s_state + order by i_item_id + ,i_item_desc + ,s_state +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.17" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF9[d_date_sk->[cs_sold_date_sk],RF7[sr_customer_sk->[cs_bill_customer_sk],RF8[sr_item_sk->[cs_item_sk],RF6[s_store_sk->[ss_store_sk],RF5[i_item_sk->[ss_item_sk],RF4[d_date_sk->[sr_returned_date_sk],RF1[ss_customer_sk->[sr_customer_sk],RF2[ss_item_sk->[sr_item_sk],RF3[ss_ticket_number->[sr_ticket_number],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf18.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf18.groovy new file mode 100644 index 000000000000000..fd59182294cbb11 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf18.groovy @@ -0,0 +1,88 @@ +/* + * 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("ds_rf18") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id, + ca_country, + ca_state, + ca_county, + avg( cast(cs_quantity as decimal(12,2))) agg1, + avg( cast(cs_list_price as decimal(12,2))) agg2, + avg( cast(cs_coupon_amt as decimal(12,2))) agg3, + avg( cast(cs_sales_price as decimal(12,2))) agg4, + avg( cast(cs_net_profit as decimal(12,2))) agg5, + avg( cast(c_birth_year as decimal(12,2))) agg6, + avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 + from catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd1.cd_demo_sk and + cs_bill_customer_sk = c_customer_sk and + cd1.cd_gender = 'F' and + cd1.cd_education_status = 'Advanced Degree' and + c_current_cdemo_sk = cd2.cd_demo_sk and + c_current_addr_sk = ca_address_sk and + c_birth_month in (10,7,8,4,1,2) and + d_year = 1998 and + ca_state in ('WA','GA','NC' + ,'ME','WY','OK','IN') + group by rollup (i_item_id, ca_country, ca_state, ca_county) + order by ca_country, + ca_state, + ca_county, + i_item_id + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.18" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[c_current_cdemo_sk->[cd_demo_sk],RF4[cs_item_sk->[i_item_sk],RF3[ca_address_sk->[c_current_addr_sk],RF2[c_customer_sk->[cs_bill_customer_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[cd_demo_sk->[cs_bill_cdemo_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf19.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf19.groovy new file mode 100644 index 000000000000000..e521f4cfb9895dc --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf19.groovy @@ -0,0 +1,79 @@ +/* + * 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("ds_rf19") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item,customer,customer_address,store + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=2 + and d_moy=12 + and d_year=1999 + and ss_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and substr(ca_zip,1,5) <> substr(s_zip,1,5) + and ss_store_sk = s_store_sk + group by i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact + order by ext_price desc + ,i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact +limit 100 ; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.19" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF4[s_store_sk->[ss_store_sk],RF3[c_current_addr_sk->[ca_address_sk],RF2[ss_customer_sk->[c_customer_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_sk->[ss_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf2.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf2.groovy new file mode 100644 index 000000000000000..2212273a7b767f7 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf2.groovy @@ -0,0 +1,114 @@ +/* + * 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("ds_rf2") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with wscs as + (select sold_date_sk + ,sales_price + from (select ws_sold_date_sk sold_date_sk + ,ws_ext_sales_price sales_price + from web_sales + union all + select cs_sold_date_sk sold_date_sk + ,cs_ext_sales_price sales_price + from catalog_sales) t), + wswscs as + (select d_week_seq, + sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales + from wscs + ,date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) + select d_week_seq1 + ,round(sun_sales1/sun_sales2,2) + ,round(mon_sales1/mon_sales2,2) + ,round(tue_sales1/tue_sales2,2) + ,round(wed_sales1/wed_sales2,2) + ,round(thu_sales1/thu_sales2,2) + ,round(fri_sales1/fri_sales2,2) + ,round(sat_sales1/sat_sales2,2) + from + (select wswscs.d_week_seq d_week_seq1 + ,sun_sales sun_sales1 + ,mon_sales mon_sales1 + ,tue_sales tue_sales1 + ,wed_sales wed_sales1 + ,thu_sales thu_sales1 + ,fri_sales fri_sales1 + ,sat_sales sat_sales1 + from wswscs,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 1998) y, + (select wswscs.d_week_seq d_week_seq2 + ,sun_sales sun_sales2 + ,mon_sales mon_sales2 + ,tue_sales tue_sales2 + ,wed_sales wed_sales2 + ,thu_sales thu_sales2 + ,fri_sales fri_sales2 + ,sat_sales sat_sales2 + from wswscs + ,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 1998+1) z + where d_week_seq1=d_week_seq2-53 + order by d_week_seq1; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.2" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF0[d_date_sk->[ws_sold_date_sk],RF1[d_date_sk->[cs_sold_date_sk],RF3[d_week_seq->[d_week_seq],RF2[d_week_seq->[d_week_seq]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf20.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf20.groovy new file mode 100644 index 000000000000000..79439a8b737f072 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf20.groovy @@ -0,0 +1,84 @@ +/* + * 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("ds_rf20") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(cs_ext_sales_price) as itemrevenue + ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over + (partition by i_class) as revenueratio + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and i_category in ('Shoes', 'Books', 'Women') + and cs_sold_date_sk = d_date_sk + and d_date between cast('2002-01-26' as date) + and (cast('2002-01-26' as date) + interval 30 day) + group by i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + order by i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.20" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[i_item_sk->[cs_item_sk],RF0[d_date_sk->[cs_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf21.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf21.groovy new file mode 100644 index 000000000000000..f11745592c0a1e3 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf21.groovy @@ -0,0 +1,84 @@ +/* + * 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("ds_rf21") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select * + from(select w_warehouse_name + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('2002-02-27' as date)) + then inv_quantity_on_hand + else 0 end) as inv_before + ,sum(case when (cast(d_date as date) >= cast ('2002-02-27' as date)) + then inv_quantity_on_hand + else 0 end) as inv_after + from inventory + ,warehouse + ,item + ,date_dim + where i_current_price between 0.99 and 1.49 + and i_item_sk = inv_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_date between (cast ('2002-02-27' as date) - interval 30 day) + and (cast ('2002-02-27' as date) + interval 30 day) + group by w_warehouse_name, i_item_id) x + where (case when inv_before > 0 + then inv_after / inv_before + else null + end) between 2.0/3.0 and 3.0/2.0 + order by w_warehouse_name + ,i_item_id + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.21" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[w_warehouse_sk->[inv_warehouse_sk],RF1[d_date_sk->[inv_date_sk],RF0[i_item_sk->[inv_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf22.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf22.groovy new file mode 100644 index 000000000000000..2a31463b5c06e6c --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf22.groovy @@ -0,0 +1,74 @@ +/* + * 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("ds_rf22") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_product_name + ,i_brand + ,i_class + ,i_category + ,avg(inv_quantity_on_hand) qoh + from inventory + ,date_dim + ,item + where inv_date_sk=d_date_sk + and inv_item_sk=i_item_sk + and d_month_seq between 1188 and 1188 + 11 + group by rollup(i_product_name + ,i_brand + ,i_class + ,i_category) +order by qoh, i_product_name, i_brand, i_class, i_category +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.22" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[inv_date_sk],RF0[i_item_sk->[inv_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf23.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf23.groovy new file mode 100644 index 000000000000000..2aea6a3f1fdb107 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf23.groovy @@ -0,0 +1,106 @@ +/* + * 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("ds_rf23") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by c_customer_sk) t), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * +from + max_store_sales)) + select sum(sales) + from (select cs_quantity*cs_list_price sales + from catalog_sales + ,date_dim + where d_year = 2000 + and d_moy = 5 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + union all + select ws_quantity*ws_list_price sales + from web_sales + ,date_dim + where d_year = 2000 + and d_moy = 5 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) t2 + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.23" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[i_item_sk->[ss_item_sk],RF0[d_date_sk->[ss_sold_date_sk],RF4[c_customer_sk->[ss_customer_sk],RF3[c_customer_sk->[ss_customer_sk],RF2[d_date_sk->[ss_sold_date_sk],RF6[c_customer_sk->[cs_bill_customer_sk],RF5[d_date_sk->[cs_sold_date_sk],RF8[c_customer_sk->[ws_bill_customer_sk],RF7[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf24.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf24.groovy new file mode 100644 index 000000000000000..1a47f501fa6c83b --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf24.groovy @@ -0,0 +1,109 @@ +/* + * 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("ds_rf24") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_net_profit) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_current_addr_sk = ca_address_sk + and c_birth_country <> upper(ca_country) + and s_zip = ca_zip +and s_market_id=8 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'beige' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +order by c_last_name + ,c_first_name + ,s_store_name +; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.24" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[sr_item_sk->[ss_item_sk],RF6[sr_ticket_number->[ss_ticket_number],RF4[i_item_sk->[ss_item_sk],RF2[ca_zip->[s_zip],RF3[ca_address_sk->[c_current_addr_sk],RF1[c_customer_sk->[ss_customer_sk],RF0[s_store_sk->[ss_store_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf25.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf25.groovy new file mode 100644 index 000000000000000..c044fb40288d3ec --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf25.groovy @@ -0,0 +1,102 @@ +/* + * 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("ds_rf25") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,sum(ss_net_profit) as store_sales_profit + ,sum(sr_net_loss) as store_returns_loss + ,sum(cs_net_profit) as catalog_sales_profit + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 2000 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 10 + and d2.d_year = 2000 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_moy between 4 and 10 + and d3.d_year = 2000 + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.25" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF9[d_date_sk->[cs_sold_date_sk],RF7[sr_customer_sk->[cs_bill_customer_sk],RF8[sr_item_sk->[cs_item_sk],RF6[s_store_sk->[ss_store_sk],RF5[i_item_sk->[ss_item_sk],RF4[d_date_sk->[sr_returned_date_sk],RF1[ss_customer_sk->[sr_customer_sk],RF2[ss_item_sk->[sr_item_sk],RF3[ss_ticket_number->[sr_ticket_number],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf26.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf26.groovy new file mode 100644 index 000000000000000..449a2f1f813651d --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf26.groovy @@ -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. + */ + +suite("ds_rf26") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 + from catalog_sales, customer_demographics, date_dim, item, promotion + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd_demo_sk and + cs_promo_sk = p_promo_sk and + cd_gender = 'M' and + cd_marital_status = 'S' and + cd_education_status = 'Unknown' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2001 + group by i_item_id + order by i_item_id + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.26" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[i_item_sk->[cs_item_sk],RF2[p_promo_sk->[cs_promo_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[cd_demo_sk->[cs_bill_cdemo_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf27.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf27.groovy new file mode 100644 index 000000000000000..82d8e6901a21733 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf27.groovy @@ -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. + */ + +suite("ds_rf27") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id, + s_state, grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, store, item + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_store_sk = s_store_sk and + ss_cdemo_sk = cd_demo_sk and + cd_gender = 'F' and + cd_marital_status = 'D' and + cd_education_status = 'Secondary' and + d_year = 1999 and + s_state in ('MO','AL', 'MI', 'TN', 'LA', 'SC') + group by rollup (i_item_id, s_state) + order by i_item_id + ,s_state + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.27" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[s_store_sk->[ss_store_sk],RF2[i_item_sk->[ss_item_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[cd_demo_sk->[ss_cdemo_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query28.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf28.groovy similarity index 50% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query28.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf28.groovy index 4147e864872e59c..b3054df262ab9bd 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query28.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf28.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -select * +/* + * 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("ds_rf28") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select * from (select avg(ss_list_price) B1_LP ,count(ss_list_price) B1_CNT ,count(distinct ss_list_price) B1_CNTD @@ -53,3 +82,26 @@ from (select avg(ss_list_price) B1_LP or ss_coupon_amt between 15257 and 15257+1000 or ss_wholesale_cost between 31 and 31+20)) B6 limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.28" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf29.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf29.groovy new file mode 100644 index 000000000000000..39ac94f02ee15c3 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf29.groovy @@ -0,0 +1,101 @@ +/* + * 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("ds_rf29") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,avg(ss_quantity) as store_sales_quantity + ,avg(sr_return_quantity) as store_returns_quantity + ,avg(cs_quantity) as catalog_sales_quantity + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 1999 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 4 + 3 + and d2.d_year = 1999 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_year in (1999,1999+1,1999+2) + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.29" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF9[ss_store_sk->[s_store_sk],RF8[ss_item_sk->[i_item_sk],RF7[cs_sold_date_sk->[d_date_sk],RF5[sr_customer_sk->[cs_bill_customer_sk],RF6[sr_item_sk->[cs_item_sk],RF4[d_date_sk->[ss_sold_date_sk],RF1[sr_customer_sk->[ss_customer_sk],RF2[sr_item_sk->[ss_item_sk],RF3[sr_ticket_number->[ss_ticket_number],RF0[d_date_sk->[sr_returned_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf3.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf3.groovy new file mode 100644 index 000000000000000..eea5867025599aa --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf3.groovy @@ -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. + */ + +suite("ds_rf3") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_sales_price) sum_agg + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 816 + and dt.d_moy=11 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,sum_agg desc + ,brand_id + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.3" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_sk->[ss_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf30.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf30.groovy new file mode 100644 index 000000000000000..a05a90cdb2c9a93 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf30.groovy @@ -0,0 +1,85 @@ +/* + * 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("ds_rf30") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with customer_total_return as + (select wr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(wr_return_amt) as ctr_total_return + from web_returns + ,date_dim + ,customer_address + where wr_returned_date_sk = d_date_sk + and d_year =2002 + and wr_returning_addr_sk = ca_address_sk + group by wr_returning_customer_sk + ,ca_state) + select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'IN' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.30" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[ca_address_sk->[wr_returning_addr_sk],RF0[d_date_sk->[wr_returned_date_sk],RF3[c_customer_sk->[ctr_customer_sk],RF2[ca_address_sk->[c_current_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf31.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf31.groovy new file mode 100644 index 000000000000000..de42ac0c39e65f8 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf31.groovy @@ -0,0 +1,106 @@ +/* + * 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("ds_rf31") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ss as + (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales + from store_sales,date_dim,customer_address + where ss_sold_date_sk = d_date_sk + and ss_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year), + ws as + (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales + from web_sales,date_dim,customer_address + where ws_sold_date_sk = d_date_sk + and ws_bill_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year) + select + ss1.ca_county + ,ss1.d_year + ,ws2.web_sales/ws1.web_sales web_q1_q2_increase + ,ss2.store_sales/ss1.store_sales store_q1_q2_increase + ,ws3.web_sales/ws2.web_sales web_q2_q3_increase + ,ss3.store_sales/ss2.store_sales store_q2_q3_increase + from + ss ss1 + ,ss ss2 + ,ss ss3 + ,ws ws1 + ,ws ws2 + ,ws ws3 + where + ss1.d_qoy = 1 + and ss1.d_year = 2000 + and ss1.ca_county = ss2.ca_county + and ss2.d_qoy = 2 + and ss2.d_year = 2000 + and ss2.ca_county = ss3.ca_county + and ss3.d_qoy = 3 + and ss3.d_year = 2000 + and ss1.ca_county = ws1.ca_county + and ws1.d_qoy = 1 + and ws1.d_year = 2000 + and ws1.ca_county = ws2.ca_county + and ws2.d_qoy = 2 + and ws2.d_year = 2000 + and ws1.ca_county = ws3.ca_county + and ws3.d_qoy = 3 + and ws3.d_year =2000 + and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end + > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end + and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end + > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end + order by web_q1_q2_increase; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.31" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[ca_address_sk->[ss_addr_sk],RF0[d_date_sk->[ss_sold_date_sk],RF3[ca_address_sk->[ws_bill_addr_sk],RF2[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf32.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf32.groovy new file mode 100644 index 000000000000000..e8aa475c1ba01c0 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf32.groovy @@ -0,0 +1,82 @@ +/* + * 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("ds_rf32") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select sum(cs_ext_discount_amt) as "excess discount amount" +from + catalog_sales + ,item + ,date_dim +where +i_manufact_id = 29 +and i_item_sk = cs_item_sk +and d_date between '1999-01-07' and + (cast('1999-01-07' as date) + interval 90 day) +and d_date_sk = cs_sold_date_sk +and cs_ext_discount_amt + > ( + select + 1.3 * avg(cs_ext_discount_amt) + from + catalog_sales + ,date_dim + where + cs_item_sk = i_item_sk + and d_date between '1999-01-07' and + (cast('1999-01-07' as date) + interval 90 day) + and d_date_sk = cs_sold_date_sk + ) +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.32" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[cs_sold_date_sk],RF0[i_item_sk->[cs_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf33.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf33.groovy new file mode 100644 index 000000000000000..88a45749b0163ec --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf33.groovy @@ -0,0 +1,129 @@ +/* + * 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("ds_rf33") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ss as ( + select + i_manufact_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Home')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2002 + and d_moy = 1 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + cs as ( + select + i_manufact_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Home')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2002 + and d_moy = 1 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + ws as ( + select + i_manufact_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Home')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2002 + and d_moy = 1 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id) + select i_manufact_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_manufact_id + order by total_sales +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.33" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[i_item_sk->[ss_item_sk],RF2[ca_address_sk->[ss_addr_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[i_manufact_id->[i_manufact_id],RF7[i_item_sk->[cs_item_sk],RF6[ca_address_sk->[cs_bill_addr_sk],RF5[d_date_sk->[cs_sold_date_sk],RF4[i_manufact_id->[i_manufact_id],RF11[i_manufact_id->[i_manufact_id],RF10[ws_item_sk->[i_item_sk],RF9[ca_address_sk->[ws_bill_addr_sk],RF8[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf34.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf34.groovy new file mode 100644 index 000000000000000..8dfbf0361536ad1 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf34.groovy @@ -0,0 +1,85 @@ +/* + * 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("ds_rf34") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) + and (household_demographics.hd_buy_potential = '1001-5000' or + household_demographics.hd_buy_potential = '0-500') + and household_demographics.hd_vehicle_count > 0 + and (case when household_demographics.hd_vehicle_count > 0 + then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count + else null + end) > 1.2 + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_county in ('Ziebach County','Daviess County','Walker County','Richland County', + 'Barrow County','Franklin Parish','Williamson County','Luce County') + group by ss_ticket_number,ss_customer_sk) dn,customer + where ss_customer_sk = c_customer_sk + and cnt between 15 and 20 + order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.34" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[ss_customer_sk->[c_customer_sk],RF2[s_store_sk->[ss_store_sk],RF1[hd_demo_sk->[ss_hdemo_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf35.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf35.groovy new file mode 100644 index 000000000000000..123775ba9018ff1 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf35.groovy @@ -0,0 +1,112 @@ +/* + * 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("ds_rf35") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + count(*) cnt1, + max(cd_dep_count), + sum(cd_dep_count), + max(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + max(cd_dep_employed_count), + sum(cd_dep_employed_count), + max(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + max(cd_dep_college_count), + sum(cd_dep_college_count), + max(cd_dep_college_count) + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2001 and + d_qoy < 4) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2001 and + d_qoy < 4) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2001 and + d_qoy < 4)) + group by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.35" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[cd_demo_sk->[c_current_cdemo_sk],RF4[ca_address_sk->[c_current_addr_sk],RF3[c_customer_sk->[ss_customer_sk],RF2[d_date_sk->[ss_sold_date_sk],RF1[d_date_sk->[ws_sold_date_sk],RF0[d_date_sk->[cs_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf36.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf36.groovy new file mode 100644 index 000000000000000..d74bcda63851bc8 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf36.groovy @@ -0,0 +1,84 @@ +/* + * 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("ds_rf36") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,item + ,store + where + d1.d_year = 2002 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and s_state in ('SD','TN','GA','SC', + 'MO','AL','MI','OH') + group by rollup(i_category,i_class) + order by + lochierarchy desc + ,case when lochierarchy = 0 then i_category end + ,rank_within_parent + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.36" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[s_store_sk->[ss_store_sk],RF1[i_item_sk->[ss_item_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf37.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf37.groovy new file mode 100644 index 000000000000000..1c0b52482b22619 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf37.groovy @@ -0,0 +1,71 @@ +/* + * 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("ds_rf37") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, catalog_sales + where i_current_price between 45 and 45 + 30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('1999-02-21' as date) and (cast('1999-02-21' as date) + interval 60 day) + and i_manufact_id in (856,707,1000,747) + and inv_quantity_on_hand between 100 and 500 + and cs_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.37" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[i_item_sk->[cs_item_sk],RF1[d_date_sk->[inv_date_sk],RF0[i_item_sk->[inv_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf38.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf38.groovy new file mode 100644 index 000000000000000..595c132d6ca8877 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf38.groovy @@ -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. + */ + +suite("ds_rf38") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select count(*) from ( + select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1183 and 1183 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1183 and 1183 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1183 and 1183 + 11 +) hot_cust +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.38" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[c_customer_sk->[ss_customer_sk],RF0[d_date_sk->[ss_sold_date_sk],RF3[c_customer_sk->[cs_bill_customer_sk],RF2[d_date_sk->[cs_sold_date_sk],RF5[c_customer_sk->[ws_bill_customer_sk],RF4[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf39.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf39.groovy new file mode 100644 index 000000000000000..2881698527f9864 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf39.groovy @@ -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. + */ + +suite("ds_rf39") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =1998 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=1 + and inv2.d_moy=1+1 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.39" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[w_warehouse_sk->[inv_warehouse_sk],RF1[i_item_sk->[inv_item_sk],RF0[d_date_sk->[inv_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query4.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf4.groovy similarity index 64% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query4.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf4.groovy index 24a0015cd324225..d3227cecc89dd33 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query4.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf4.groovy @@ -1,7 +1,38 @@ -explain shape plan - - -with year_total as ( +/* + * 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("ds_rf4") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with year_total as ( select c_customer_id customer_id ,c_first_name customer_first_name ,c_last_name customer_last_name @@ -114,3 +145,26 @@ union all ,t_s_secyear.customer_last_name ,t_s_secyear.customer_birth_country limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.4" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[ss_sold_date_sk],RF0[c_customer_sk->[ss_customer_sk],RF3[d_date_sk->[cs_sold_date_sk],RF2[c_customer_sk->[cs_bill_customer_sk],RF5[d_date_sk->[ws_sold_date_sk],RF4[c_customer_sk->[ws_bill_customer_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf40.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf40.groovy new file mode 100644 index 000000000000000..0d24260fe52fdfb --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf40.groovy @@ -0,0 +1,82 @@ +/* + * 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("ds_rf40") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + w_state + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('2001-04-02' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before + ,sum(case when (cast(d_date as date) >= cast ('2001-04-02' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after + from + catalog_sales left outer join catalog_returns on + (cs_order_number = cr_order_number + and cs_item_sk = cr_item_sk) + ,warehouse + ,item + ,date_dim + where + i_current_price between 0.99 and 1.49 + and i_item_sk = cs_item_sk + and cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and d_date between (cast ('2001-04-02' as date) - interval 30 day) + and (cast ('2001-04-02' as date) + interval 30 day) + group by + w_state,i_item_id + order by w_state,i_item_id +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.40" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF4[w_warehouse_sk->[cs_warehouse_sk],RF2[cs_order_number->[cr_order_number],RF3[cs_item_sk->[cr_item_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[i_item_sk->[cs_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf41.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf41.groovy new file mode 100644 index 000000000000000..54c0369f0a475d3 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf41.groovy @@ -0,0 +1,106 @@ +/* + * 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("ds_rf41") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select distinct(i_product_name) + from item i1 + where i_manufact_id between 748 and 748+40 + and (select count(*) as item_cnt + from item + where (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'gainsboro' or i_color = 'aquamarine') and + (i_units = 'Ounce' or i_units = 'Dozen') and + (i_size = 'medium' or i_size = 'economy') + ) or + (i_category = 'Women' and + (i_color = 'chiffon' or i_color = 'violet') and + (i_units = 'Ton' or i_units = 'Pound') and + (i_size = 'extra large' or i_size = 'small') + ) or + (i_category = 'Men' and + (i_color = 'chartreuse' or i_color = 'blue') and + (i_units = 'Each' or i_units = 'Oz') and + (i_size = 'N/A' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'tan' or i_color = 'dodger') and + (i_units = 'Bunch' or i_units = 'Tsp') and + (i_size = 'medium' or i_size = 'economy') + ))) or + (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'blanched' or i_color = 'tomato') and + (i_units = 'Tbl' or i_units = 'Case') and + (i_size = 'medium' or i_size = 'economy') + ) or + (i_category = 'Women' and + (i_color = 'almond' or i_color = 'lime') and + (i_units = 'Box' or i_units = 'Dram') and + (i_size = 'extra large' or i_size = 'small') + ) or + (i_category = 'Men' and + (i_color = 'peru' or i_color = 'saddle') and + (i_units = 'Pallet' or i_units = 'Gram') and + (i_size = 'N/A' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'indian' or i_color = 'spring') and + (i_units = 'Unknown' or i_units = 'Carton') and + (i_size = 'medium' or i_size = 'economy') + )))) > 0 + order by i_product_name + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.41" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF0[i_manufact->[i_manufact]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf42.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf42.groovy new file mode 100644 index 000000000000000..0e478423e079556 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf42.groovy @@ -0,0 +1,76 @@ +/* + * 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("ds_rf42") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select dt.d_year + ,item.i_category_id + ,item.i_category + ,sum(ss_ext_sales_price) + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=11 + and dt.d_year=2002 + group by dt.d_year + ,item.i_category_id + ,item.i_category + order by sum(ss_ext_sales_price) desc,dt.d_year + ,item.i_category_id + ,item.i_category +limit 100 ; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.42" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_sk->[ss_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf43.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf43.groovy new file mode 100644 index 000000000000000..e7da261b03cd816 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf43.groovy @@ -0,0 +1,73 @@ +/* + * 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("ds_rf43") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select s_store_name, s_store_id, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from date_dim, store_sales, store + where d_date_sk = ss_sold_date_sk and + s_store_sk = ss_store_sk and + s_gmt_offset = -5 and + d_year = 2000 + group by s_store_name, s_store_id + order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.43" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[s_store_sk->[ss_store_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf44.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf44.groovy new file mode 100644 index 000000000000000..f626136021a0d2d --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf44.groovy @@ -0,0 +1,89 @@ +/* + * 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("ds_rf44") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing +from(select * + from (select item_sk,rank() over (order by rank_col asc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 146 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 146 + and ss_addr_sk is null + group by ss_store_sk))V1)V11 + where rnk < 11) asceding, + (select * + from (select item_sk,rank() over (order by rank_col desc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 146 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 146 + and ss_addr_sk is null + group by ss_store_sk))V2)V21 + where rnk < 11) descending, +item i1, +item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk=asceding.item_sk + and i2.i_item_sk=descending.item_sk +order by asceding.rnk +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.44" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[item_sk->[i_item_sk],RF0[item_sk->[i_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf45.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf45.groovy new file mode 100644 index 000000000000000..15d79d0e57a244f --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf45.groovy @@ -0,0 +1,74 @@ +/* + * 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("ds_rf45") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select ca_zip, ca_city, sum(ws_sales_price) + from web_sales, customer, customer_address, date_dim, item + where ws_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ws_item_sk = i_item_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') + or + i_item_id in (select i_item_id + from item + where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) + ) + and ws_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 2000 + group by ca_zip, ca_city + order by ca_zip, ca_city + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.45" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[c_customer_sk->[ws_bill_customer_sk],RF2[i_item_sk->[ws_item_sk],RF1[d_date_sk->[ws_sold_date_sk],RF0[ca_address_sk->[c_current_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf46.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf46.groovy new file mode 100644 index 000000000000000..796ce479e4635dc --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf46.groovy @@ -0,0 +1,89 @@ +/* + * 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("ds_rf46") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and (household_demographics.hd_dep_count = 6 or + household_demographics.hd_vehicle_count= 0) + and date_dim.d_dow in (6,0) + and date_dim.d_year in (1999,1999+1,1999+2) + and store.s_city in ('Five Points','Centerville','Oak Grove','Fairview','Liberty') + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.46" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[c_customer_sk->[ss_customer_sk],RF4[ca_address_sk->[ss_addr_sk],RF3[s_store_sk->[ss_store_sk],RF2[hd_demo_sk->[ss_hdemo_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[c_current_addr_sk->[ca_address_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf47.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf47.groovy new file mode 100644 index 000000000000000..769227e276e6db6 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf47.groovy @@ -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. + */ + +suite("ds_rf47") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with v1 as( + select i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + s_store_name, s_company_name + order by d_year, d_moy) rn + from item, store_sales, date_dim, store + where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + ( + d_year = 2001 or + ( d_year = 2001-1 and d_moy =12) or + ( d_year = 2001+1 and d_moy =1) + ) + group by i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 as( + select v1.s_store_name + ,v1.d_year + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1.s_store_name = v1_lag.s_store_name and + v1.s_store_name = v1_lead.s_store_name and + v1.s_company_name = v1_lag.s_company_name and + v1.s_company_name = v1_lead.s_company_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2001 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, nsum + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.47" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[s_store_sk->[ss_store_sk],RF1[i_item_sk->[ss_item_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf48.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf48.groovy new file mode 100644 index 000000000000000..32f767349fd6a19 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf48.groovy @@ -0,0 +1,121 @@ +/* + * 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("ds_rf48") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select sum (ss_quantity) + from store_sales, store, customer_demographics, customer_address, date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 1999 + and + ( + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'U' + and + cd_education_status = 'Primary' + and + ss_sales_price between 100.00 and 150.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'W' + and + cd_education_status = 'College' + and + ss_sales_price between 50.00 and 100.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'D' + and + cd_education_status = '2 yr Degree' + and + ss_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('MD', 'MN', 'IA') + and ss_net_profit between 0 and 2000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('VA', 'IL', 'TX') + and ss_net_profit between 150 and 3000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('MI', 'WI', 'IN') + and ss_net_profit between 50 and 25000 + ) + ) +; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.48" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[s_store_sk->[ss_store_sk],RF2[d_date_sk->[ss_sold_date_sk],RF1[cd_demo_sk->[ss_cdemo_sk],RF0[ca_address_sk->[ss_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query49.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf49.groovy similarity index 59% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query49.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf49.groovy index ec3fd6693c4e8d0..0f85f343d8eba07 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query49.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf49.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -select channel, item, return_ratio, return_rank, currency_rank from +/* + * 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("ds_rf49") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select channel, item, return_ratio, return_rank, currency_rank from (select 'web' as channel ,web.item @@ -129,3 +158,26 @@ select channel, item, return_ratio, return_rank, currency_rank from ) t order by 1,4,5,2 limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.49" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[ws_order_number->[wr_order_number],RF2[item->[wr_item_sk],RF0[d_date_sk->[ws_sold_date_sk],RF4[cs_order_number->[cr_order_number],RF5[item->[cr_item_sk],RF3[d_date_sk->[cs_sold_date_sk],RF7[ss_ticket_number->[sr_ticket_number],RF8[item->[sr_item_sk],RF6[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query5.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf5.groovy similarity index 60% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query5.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf5.groovy index 304682364acd959..8803e3b1e9f88c7 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query5.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf5.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -with ssr as +/* + * 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("ds_rf5") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ssr as (select s_store_id, sum(sales_price) as sales, sum(profit) as profit, @@ -128,3 +157,26 @@ with ssr as order by channel ,id limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.5" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[s_store_sk->[ss_store_sk],RF3[s_store_sk->[sr_store_sk],RF0[d_date_sk->[ss_sold_date_sk],RF1[d_date_sk->[sr_returned_date_sk],RF4[d_date_sk->[cs_sold_date_sk],RF5[d_date_sk->[cr_returned_date_sk],RF10[web_site_sk->[ws_web_site_sk],RF11[web_site_sk->[ws_web_site_sk],RF8[d_date_sk->[ws_sold_date_sk],RF9[d_date_sk->[wr_returned_date_sk],RF6[wr_item_sk->[ws_item_sk],RF7[wr_order_number->[ws_order_number]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf50.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf50.groovy new file mode 100644 index 000000000000000..529dca01febc7df --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf50.groovy @@ -0,0 +1,113 @@ +/* + * 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("ds_rf50") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and + (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and + (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and + (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + store_sales + ,store_returns + ,store + ,date_dim d1 + ,date_dim d2 +where + d2.d_year = 2001 +and d2.d_moy = 8 +and ss_ticket_number = sr_ticket_number +and ss_item_sk = sr_item_sk +and ss_sold_date_sk = d1.d_date_sk +and sr_returned_date_sk = d2.d_date_sk +and ss_customer_sk = sr_customer_sk +and ss_store_sk = s_store_sk +group by + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +order by s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.50" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[s_store_sk->[ss_store_sk],RF4[ss_sold_date_sk->[d_date_sk],RF1[sr_customer_sk->[ss_customer_sk],RF2[sr_item_sk->[ss_item_sk],RF3[sr_ticket_number->[ss_ticket_number],RF0[d_date_sk->[sr_returned_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf51.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf51.groovy new file mode 100644 index 000000000000000..28ea4237cfa561b --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf51.groovy @@ -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. + */ + +suite("ds_rf51") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + WITH web_v1 as ( +select + ws_item_sk item_sk, d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from web_sales + ,date_dim +where ws_sold_date_sk=d_date_sk + and d_month_seq between 1216 and 1216+11 + and ws_item_sk is not NULL +group by ws_item_sk, d_date), +store_v1 as ( +select + ss_item_sk item_sk, d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from store_sales + ,date_dim +where ss_sold_date_sk=d_date_sk + and d_month_seq between 1216 and 1216+11 + and ss_item_sk is not NULL +group by ss_item_sk, d_date) + select * +from (select item_sk + ,d_date + ,web_sales + ,store_sales + ,max(web_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative + ,max(store_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative + from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk + ,case when web.d_date is not null then web.d_date else store.d_date end d_date + ,web.cume_sales web_sales + ,store.cume_sales store_sales + from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + )x )y +where web_cumulative > store_cumulative +order by item_sk + ,d_date +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.51" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[ss_sold_date_sk],RF0[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf52.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf52.groovy new file mode 100644 index 000000000000000..8967a32396d4d70 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf52.groovy @@ -0,0 +1,76 @@ +/* + * 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("ds_rf52") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_ext_sales_price) ext_price + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=12 + and dt.d_year=2002 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,ext_price desc + ,brand_id +limit 100 ; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.52" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_sk->[ss_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf53.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf53.groovy new file mode 100644 index 000000000000000..e74d96cf475cce7 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf53.groovy @@ -0,0 +1,82 @@ +/* + * 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("ds_rf53") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select * from +(select i_manufact_id, +sum(ss_sales_price) sum_sales, +avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and +ss_sold_date_sk = d_date_sk and +ss_store_sk = s_store_sk and +d_month_seq in (1200,1200+1,1200+2,1200+3,1200+4,1200+5,1200+6,1200+7,1200+8,1200+9,1200+10,1200+11) and +((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'))) +group by i_manufact_id, d_qoy ) tmp1 +where case when avg_quarterly_sales > 0 + then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales + else null end > 0.1 +order by avg_quarterly_sales, + sum_sales, + i_manufact_id +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.53" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[s_store_sk->[ss_store_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_sk->[ss_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf54.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf54.groovy new file mode 100644 index 000000000000000..ccba661af7b0d3e --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf54.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("ds_rf54") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with my_customers as ( + select distinct c_customer_sk + , c_current_addr_sk + from + ( select cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + from catalog_sales + union all + select ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + from web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + where sold_date_sk = d_date_sk + and item_sk = i_item_sk + and i_category = 'Women' + and i_class = 'maternity' + and c_customer_sk = cs_or_ws_sales.customer_sk + and d_moy = 5 + and d_year = 1998 + ) + , my_revenue as ( + select c_customer_sk, + sum(ss_ext_sales_price) as revenue + from my_customers, + store_sales, + customer_address, + store, + date_dim + where c_current_addr_sk = ca_address_sk + and ca_county = s_county + and ca_state = s_state + and ss_sold_date_sk = d_date_sk + and c_customer_sk = ss_customer_sk + and d_month_seq between (select distinct d_month_seq+1 + from date_dim where d_year = 1998 and d_moy = 5) + and (select distinct d_month_seq+3 + from date_dim where d_year = 1998 and d_moy = 5) + group by c_customer_sk + ) + , segments as + (select cast((revenue/50) as int) as segment + from my_revenue + ) + select segment, count(*) as num_customers, segment*50 as segment_base + from segments + group by segment + order by segment, num_customers + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.54" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF6[s_county->[ca_county],RF7[s_state->[ca_state],RF5[d_date_sk->[ss_sold_date_sk],RF4[c_customer_sk->[ss_customer_sk],RF3[c_current_addr_sk->[ca_address_sk],RF2[customer_sk->[c_customer_sk],RF0[i_item_sk->[cs_item_sk],RF1[i_item_sk->[ws_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf55.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf55.groovy new file mode 100644 index 000000000000000..90fd6e30b7e7a54 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf55.groovy @@ -0,0 +1,68 @@ +/* + * 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("ds_rf55") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_brand_id brand_id, i_brand brand, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=100 + and d_moy=12 + and d_year=2000 + group by i_brand, i_brand_id + order by ext_price desc, i_brand_id +limit 100 ; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.55" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_sk->[ss_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf56.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf56.groovy new file mode 100644 index 000000000000000..49a3ec255de55d9 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf56.groovy @@ -0,0 +1,123 @@ +/* + * 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("ds_rf56") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ss as ( + select i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where i_item_id in (select + i_item_id +from item +where i_color in ('powder','green','cyan')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 2 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + cs as ( + select i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('powder','green','cyan')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 2 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + ws as ( + select i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('powder','green','cyan')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 2 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id) + select i_item_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by total_sales, + i_item_id + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.56" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[ca_address_sk->[ss_addr_sk],RF2[i_item_sk->[ss_item_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_id->[i_item_id],RF7[ca_address_sk->[cs_bill_addr_sk],RF6[i_item_sk->[cs_item_sk],RF5[d_date_sk->[cs_sold_date_sk],RF4[i_item_id->[i_item_id],RF11[ws_bill_addr_sk->[ca_address_sk],RF10[i_item_sk->[ws_item_sk],RF9[d_date_sk->[ws_sold_date_sk],RF8[i_item_id->[i_item_id]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf57.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf57.groovy new file mode 100644 index 000000000000000..4633893be3e57ce --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf57.groovy @@ -0,0 +1,102 @@ +/* + * 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("ds_rf57") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with v1 as( + select i_category, i_brand, + cc_name, + d_year, d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) over + (partition by i_category, i_brand, + cc_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + cc_name + order by d_year, d_moy) rn + from item, catalog_sales, date_dim, call_center + where cs_item_sk = i_item_sk and + cs_sold_date_sk = d_date_sk and + cc_call_center_sk= cs_call_center_sk and + ( + d_year = 1999 or + ( d_year = 1999-1 and d_moy =12) or + ( d_year = 1999+1 and d_moy =1) + ) + group by i_category, i_brand, + cc_name , d_year, d_moy), + v2 as( + select v1.i_brand + ,v1.d_year + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1. cc_name = v1_lag. cc_name and + v1. cc_name = v1_lead. cc_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 1999 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, nsum + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.57" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[cc_call_center_sk->[cs_call_center_sk],RF1[i_item_sk->[cs_item_sk],RF0[d_date_sk->[cs_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf58.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf58.groovy new file mode 100644 index 000000000000000..b3d7edb5453628c --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf58.groovy @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +suite("ds_rf58") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ss_items as + (select i_item_id item_id + ,sum(ss_ext_sales_price) ss_item_rev + from store_sales + ,item + ,date_dim + where ss_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '2001-03-24')) + and ss_sold_date_sk = d_date_sk + group by i_item_id), + cs_items as + (select i_item_id item_id + ,sum(cs_ext_sales_price) cs_item_rev + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '2001-03-24')) + and cs_sold_date_sk = d_date_sk + group by i_item_id), + ws_items as + (select i_item_id item_id + ,sum(ws_ext_sales_price) ws_item_rev + from web_sales + ,item + ,date_dim + where ws_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq =(select d_week_seq + from date_dim + where d_date = '2001-03-24')) + and ws_sold_date_sk = d_date_sk + group by i_item_id) + select ss_items.item_id + ,ss_item_rev + ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev + ,cs_item_rev + ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev + ,ws_item_rev + ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev + ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average + from ss_items,cs_items,ws_items + where ss_items.item_id=cs_items.item_id + and ss_items.item_id=ws_items.item_id + and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + order by item_id + ,ss_item_rev + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.58" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF13[item_id->[i_item_id],RF12[d_date->[d_date],RF11[d_date_sk->[cs_sold_date_sk],RF10[i_item_sk->[cs_item_sk],RF9[d_week_seq->[d_week_seq],RF8[item_id->[i_item_id],RF7[d_date->[d_date],RF6[d_date_sk->[ss_sold_date_sk],RF5[i_item_sk->[ss_item_sk],RF4[d_week_seq->[d_week_seq],RF3[d_date->[d_date],RF2[d_date_sk->[ws_sold_date_sk],RF1[i_item_sk->[ws_item_sk],RF0[d_week_seq->[d_week_seq]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf59.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf59.groovy new file mode 100644 index 000000000000000..f6d114d47807631 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf59.groovy @@ -0,0 +1,98 @@ +/* + * 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("ds_rf59") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with wss as + (select d_week_seq, + ss_store_sk, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + group by d_week_seq,ss_store_sk + ) + select s_store_name1,s_store_id1,d_week_seq1 + ,sun_sales1/sun_sales2,mon_sales1/mon_sales2 + ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2 + ,fri_sales1/fri_sales2,sat_sales1/sat_sales2 + from + (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1 + ,s_store_id s_store_id1,sun_sales sun_sales1 + ,mon_sales mon_sales1,tue_sales tue_sales1 + ,wed_sales wed_sales1,thu_sales thu_sales1 + ,fri_sales fri_sales1,sat_sales sat_sales1 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1196 and 1196 + 11) y, + (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2 + ,s_store_id s_store_id2,sun_sales sun_sales2 + ,mon_sales mon_sales2,tue_sales tue_sales2 + ,wed_sales wed_sales2,thu_sales thu_sales2 + ,fri_sales fri_sales2,sat_sales sat_sales2 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1196+ 12 and 1196 + 23) x + where s_store_id1=s_store_id2 + and d_week_seq1=d_week_seq2-52 + order by s_store_name1,s_store_id1,d_week_seq1 +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.59" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF0[d_date_sk->[ss_sold_date_sk],RF4[s_store_sk->[ss_store_sk],RF3[d_week_seq->[d_week_seq],RF2[s_store_sk->[ss_store_sk],RF1[d_week_seq->[d_week_seq]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf6.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf6.groovy new file mode 100644 index 000000000000000..1a7206e4a0f273e --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf6.groovy @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +suite("ds_rf6") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select a.ca_state state, count(*) cnt + from customer_address a + ,customer c + ,store_sales s + ,date_dim d + ,item i + where a.ca_address_sk = c.c_current_addr_sk + and c.c_customer_sk = s.ss_customer_sk + and s.ss_sold_date_sk = d.d_date_sk + and s.ss_item_sk = i.i_item_sk + and d.d_month_seq = + (select distinct (d_month_seq) + from date_dim + where d_year = 2002 + and d_moy = 3 ) + and i.i_current_price > 1.2 * + (select avg(j.i_current_price) + from item j + where j.i_category = i.i_category) + group by a.ca_state + having count(*) >= 10 + order by cnt, a.ca_state + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.6" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[i_category->[i_category],RF4[i_item_sk->[ss_item_sk],RF3[d_month_seq->[d_month_seq],RF2[d_date_sk->[ss_sold_date_sk],RF1[c_customer_sk->[ss_customer_sk],RF0[ca_address_sk->[c_current_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf60.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf60.groovy new file mode 100644 index 000000000000000..22c6361f48730c6 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf60.groovy @@ -0,0 +1,132 @@ +/* + * 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("ds_rf60") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ss as ( + select + i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 8 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -7 + group by i_item_id), + cs as ( + select + i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 8 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -7 + group by i_item_id), + ws as ( + select + i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 8 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -7 + group by i_item_id) + select + i_item_id +,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by i_item_id + ,total_sales + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.60" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[i_item_sk->[ss_item_sk],RF2[ca_address_sk->[ss_addr_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_id->[i_item_id],RF7[i_item_sk->[cs_item_sk],RF6[ca_address_sk->[cs_bill_addr_sk],RF5[d_date_sk->[cs_sold_date_sk],RF4[i_item_id->[i_item_id],RF11[i_item_sk->[ws_item_sk],RF10[ca_address_sk->[ws_bill_addr_sk],RF9[d_date_sk->[ws_sold_date_sk],RF8[i_item_id->[i_item_id]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf61.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf61.groovy new file mode 100644 index 000000000000000..600c6b401427332 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf61.groovy @@ -0,0 +1,98 @@ +/* + * 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("ds_rf61") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100 +from + (select sum(ss_ext_sales_price) promotions + from store_sales + ,store + ,promotion + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_promo_sk = p_promo_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Jewelry' + and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') + and s_gmt_offset = -7 + and d_year = 1999 + and d_moy = 11) promotional_sales, + (select sum(ss_ext_sales_price) total + from store_sales + ,store + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Jewelry' + and s_gmt_offset = -7 + and d_year = 1999 + and d_moy = 11) all_sales +order by promotions, total +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.61" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF10[c_current_addr_sk->[ca_address_sk],RF9[ss_customer_sk->[c_customer_sk],RF8[ss_item_sk->[i_item_sk],RF7[p_promo_sk->[ss_promo_sk],RF6[d_date_sk->[ss_sold_date_sk],RF5[s_store_sk->[ss_store_sk],RF4[c_current_addr_sk->[ca_address_sk],RF3[ss_customer_sk->[c_customer_sk],RF2[ss_item_sk->[i_item_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[s_store_sk->[ss_store_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf62.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf62.groovy new file mode 100644 index 000000000000000..8a044331c36166a --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf62.groovy @@ -0,0 +1,89 @@ +/* + * 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("ds_rf62") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and + (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and + (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and + (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + web_sales + ,warehouse + ,ship_mode + ,web_site + ,date_dim +where + d_month_seq between 1194 and 1194 + 11 +and ws_ship_date_sk = d_date_sk +and ws_warehouse_sk = w_warehouse_sk +and ws_ship_mode_sk = sm_ship_mode_sk +and ws_web_site_sk = web_site_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.62" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[w_warehouse_sk->[ws_warehouse_sk],RF2[sm_ship_mode_sk->[ws_ship_mode_sk],RF1[web_site_sk->[ws_web_site_sk],RF0[d_date_sk->[ws_ship_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf63.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf63.groovy new file mode 100644 index 000000000000000..55148d4a047351b --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf63.groovy @@ -0,0 +1,83 @@ +/* + * 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("ds_rf63") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select * +from (select i_manager_id + ,sum(ss_sales_price) sum_sales + ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales + from item + ,store_sales + ,date_dim + ,store + where ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and d_month_seq in (1181,1181+1,1181+2,1181+3,1181+4,1181+5,1181+6,1181+7,1181+8,1181+9,1181+10,1181+11) + and (( 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'))) +group by i_manager_id, d_moy) tmp1 +where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 +order by i_manager_id + ,avg_monthly_sales + ,sum_sales +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.63" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[s_store_sk->[ss_store_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_sk->[ss_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf64.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf64.groovy new file mode 100644 index 000000000000000..2542bf98e39934f --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf64.groovy @@ -0,0 +1,175 @@ +/* + * 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("ds_rf64") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with cs_ui as + (select cs_item_sk + ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund + from catalog_sales + ,catalog_returns + where cs_item_sk = cr_item_sk + and cs_order_number = cr_order_number + group by cs_item_sk + having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)), +cross_sales as + (select i_product_name product_name + ,i_item_sk item_sk + ,s_store_name store_name + ,s_zip store_zip + ,ad1.ca_street_number b_street_number + ,ad1.ca_street_name b_street_name + ,ad1.ca_city b_city + ,ad1.ca_zip b_zip + ,ad2.ca_street_number c_street_number + ,ad2.ca_street_name c_street_name + ,ad2.ca_city c_city + ,ad2.ca_zip c_zip + ,d1.d_year as syear + ,d2.d_year as fsyear + ,d3.d_year s2year + ,count(*) cnt + ,sum(ss_wholesale_cost) s1 + ,sum(ss_list_price) s2 + ,sum(ss_coupon_amt) s3 + FROM store_sales + ,store_returns + ,cs_ui + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,customer + ,customer_demographics cd1 + ,customer_demographics cd2 + ,promotion + ,household_demographics hd1 + ,household_demographics hd2 + ,customer_address ad1 + ,customer_address ad2 + ,income_band ib1 + ,income_band ib2 + ,item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk= cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk and + ss_item_sk = i_item_sk and + ss_item_sk = sr_item_sk and + ss_ticket_number = sr_ticket_number and + ss_item_sk = cs_ui.cs_item_sk and + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk and + c_first_sales_date_sk = d2.d_date_sk and + c_first_shipto_date_sk = d3.d_date_sk and + ss_promo_sk = p_promo_sk and + hd1.hd_income_band_sk = ib1.ib_income_band_sk and + hd2.hd_income_band_sk = ib2.ib_income_band_sk and + cd1.cd_marital_status <> cd2.cd_marital_status and + i_color in ('blanched','medium','brown','chocolate','burlywood','drab') and + i_current_price between 23 and 23 + 10 and + i_current_price between 23 + 1 and 23 + 15 +group by i_product_name + ,i_item_sk + ,s_store_name + ,s_zip + ,ad1.ca_street_number + ,ad1.ca_street_name + ,ad1.ca_city + ,ad1.ca_zip + ,ad2.ca_street_number + ,ad2.ca_street_name + ,ad2.ca_city + ,ad2.ca_zip + ,d1.d_year + ,d2.d_year + ,d3.d_year +) +select cs1.product_name + ,cs1.store_name + ,cs1.store_zip + ,cs1.b_street_number + ,cs1.b_street_name + ,cs1.b_city + ,cs1.b_zip + ,cs1.c_street_number + ,cs1.c_street_name + ,cs1.c_city + ,cs1.c_zip + ,cs1.syear + ,cs1.cnt + ,cs1.s1 as s11 + ,cs1.s2 as s21 + ,cs1.s3 as s31 + ,cs2.s1 as s12 + ,cs2.s2 as s22 + ,cs2.s3 as s32 + ,cs2.syear + ,cs2.cnt +from cross_sales cs1,cross_sales cs2 +where cs1.item_sk=cs2.item_sk and + cs1.syear = 2001 and + cs2.syear = 2001 + 1 and + cs2.cnt <= cs1.cnt and + cs1.store_name = cs2.store_name and + cs1.store_zip = cs2.store_zip +order by cs1.product_name + ,cs1.store_name + ,cs2.cnt + ,cs1.s1 + ,cs2.s1; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.64" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + //assertEquals("RF19[c_first_shipto_date_sk->[d_date_sk],RF18[c_current_addr_sk->[ca_address_sk],RF17[ss_item_sk->[cs_item_sk],RF15[cr_order_number->[cs_order_number],RF16[cr_item_sk->[cs_item_sk],RF14[hd_income_band_sk->[ib_income_band_sk],RF13[c_first_sales_date_sk->[d_date_sk],RF12[c_current_hdemo_sk->[hd_demo_sk],RF11[ss_promo_sk->[p_promo_sk],RF10[hd_income_band_sk->[ib_income_band_sk],RF8[ss_item_sk->[sr_item_sk],RF9[ss_ticket_number->[sr_ticket_number],RF7[c_current_cdemo_sk->[cd_demo_sk],RF6[ss_cdemo_sk->[cd_demo_sk],RF5[d_date_sk->[ss_sold_date_sk],RF4[hd_demo_sk->[ss_hdemo_sk],RF3[ss_customer_sk->[c_customer_sk],RF2[s_store_sk->[ss_store_sk],RF1[ca_address_sk->[ss_addr_sk],RF0[item_sk->[ss_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf65.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf65.groovy new file mode 100644 index 000000000000000..ea0856c3030f931 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf65.groovy @@ -0,0 +1,83 @@ +/* + * 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("ds_rf65") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand + from store, item, + (select ss_store_sk, avg(revenue) as ave + from + (select ss_store_sk, ss_item_sk, + sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1221 and 1221+11 + group by ss_store_sk, ss_item_sk) sa + group by ss_store_sk) sb, + (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1221 and 1221+11 + group by ss_store_sk, ss_item_sk) sc + where sb.ss_store_sk = sc.ss_store_sk and + sc.revenue <= 0.1 * sb.ave and + s_store_sk = sc.ss_store_sk and + i_item_sk = sc.ss_item_sk + order by s_store_name, i_item_desc +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.65" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF4[s_store_sk->[ss_store_sk],RF3[i_item_sk->[ss_item_sk],RF2[ss_store_sk->[ss_store_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query66.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf66.groovy similarity index 76% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query66.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf66.groovy index 66fe66328ff851e..d6011f9c721e0ad 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query66.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf66.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -select +/* + * 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("ds_rf66") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select w_warehouse_name ,w_warehouse_sq_ft ,w_city @@ -220,3 +249,26 @@ select ,year order by w_warehouse_name limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.66" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[w_warehouse_sk->[ws_warehouse_sk],RF2[t_time_sk->[ws_sold_time_sk],RF1[d_date_sk->[ws_sold_date_sk],RF0[sm_ship_mode_sk->[ws_ship_mode_sk],RF7[w_warehouse_sk->[cs_warehouse_sk],RF6[t_time_sk->[cs_sold_time_sk],RF5[d_date_sk->[cs_sold_date_sk],RF4[sm_ship_mode_sk->[cs_ship_mode_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf67.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf67.groovy new file mode 100644 index 000000000000000..e3dd38c7391556d --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf67.groovy @@ -0,0 +1,98 @@ +/* + * 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("ds_rf67") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select * +from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rank() over (partition by i_category order by sumsales desc) rk + from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales + from store_sales + ,date_dim + ,store + ,item + where ss_sold_date_sk=d_date_sk + and ss_item_sk=i_item_sk + and ss_store_sk = s_store_sk + and d_month_seq between 1206 and 1206+11 + group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2 +where rk <= 100 +order by i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rk +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.67" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[s_store_sk->[ss_store_sk],RF1[i_item_sk->[ss_item_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf68.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf68.groovy new file mode 100644 index 000000000000000..6acaed55b9ba685 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf68.groovy @@ -0,0 +1,96 @@ +/* + * 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("ds_rf68") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,extended_price + ,extended_tax + ,list_price + from (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_ext_sales_price) extended_price + ,sum(ss_ext_list_price) list_price + ,sum(ss_ext_tax) extended_tax + from store_sales + ,date_dim + ,store + ,household_demographics + ,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_dep_count = 8 or + household_demographics.hd_vehicle_count= -1) + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_city in ('Pleasant Hill','Five Points') + group by ss_ticket_number + ,ss_customer_sk + ,ss_addr_sk,ca_city) dn + ,customer + ,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,ss_ticket_number + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.68" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[c_current_addr_sk->[ca_address_sk],RF4[ss_customer_sk->[c_customer_sk],RF3[ss_addr_sk->[ca_address_sk],RF2[hd_demo_sk->[ss_hdemo_sk],RF1[s_store_sk->[ss_store_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf69.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf69.groovy new file mode 100644 index 000000000000000..b6990127826b80c --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf69.groovy @@ -0,0 +1,101 @@ +/* + * 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("ds_rf69") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_state in ('TX','VA','MI') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2000 and + d_moy between 1 and 1+2) and + (not exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2000 and + d_moy between 1 and 1+2) and + not exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2000 and + d_moy between 1 and 1+2)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.69" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF7[c_customer_sk->[ss_customer_sk],RF6[d_date_sk->[ss_sold_date_sk],RF5[c_customer_sk->[cs_ship_customer_sk],RF4[d_date_sk->[cs_sold_date_sk],RF3[c_current_cdemo_sk->[cd_demo_sk],RF2[c_customer_sk->[ws_bill_customer_sk],RF1[d_date_sk->[ws_sold_date_sk],RF0[ca_address_sk->[c_current_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf7.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf7.groovy new file mode 100644 index 000000000000000..4df10c25ea70581 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf7.groovy @@ -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. + */ + +suite("ds_rf7") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, item, promotion + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_cdemo_sk = cd_demo_sk and + ss_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'College' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2001 + group by i_item_id + order by i_item_id + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.7" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[i_item_sk->[ss_item_sk],RF2[p_promo_sk->[ss_promo_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[cd_demo_sk->[ss_cdemo_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf70.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf70.groovy new file mode 100644 index 000000000000000..f0e55374d3f76cb --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf70.groovy @@ -0,0 +1,92 @@ +/* + * 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("ds_rf70") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + sum(ss_net_profit) as total_sum + ,s_state + ,s_county + ,grouping(s_state)+grouping(s_county) as lochierarchy + ,rank() over ( + partition by grouping(s_state)+grouping(s_county), + case when grouping(s_county) = 0 then s_state end + order by sum(ss_net_profit) desc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,store + where + d1.d_month_seq between 1213 and 1213+11 + and d1.d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + and s_state in + ( select s_state + from (select s_state as s_state, + rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking + from store_sales, store, date_dim + where d_month_seq between 1213 and 1213+11 + and d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + group by s_state + ) tmp1 + where ranking <= 5 + ) + group by rollup(s_state,s_county) + order by + lochierarchy desc + ,case when lochierarchy = 0 then s_state end + ,rank_within_parent + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.70" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF4[s_store_sk->[ss_store_sk],RF3[d_date_sk->[ss_sold_date_sk],RF2[s_state->[s_state],RF1[s_store_sk->[ss_store_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf71.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf71.groovy new file mode 100644 index 000000000000000..27b946f62112fff --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf71.groovy @@ -0,0 +1,94 @@ +/* + * 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("ds_rf71") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_brand_id brand_id, i_brand brand,t_hour,t_minute, + sum(ext_price) ext_price + from item, (select ws_ext_sales_price as ext_price, + ws_sold_date_sk as sold_date_sk, + ws_item_sk as sold_item_sk, + ws_sold_time_sk as time_sk + from web_sales,date_dim + where d_date_sk = ws_sold_date_sk + and d_moy=12 + and d_year=1998 + union all + select cs_ext_sales_price as ext_price, + cs_sold_date_sk as sold_date_sk, + cs_item_sk as sold_item_sk, + cs_sold_time_sk as time_sk + from catalog_sales,date_dim + where d_date_sk = cs_sold_date_sk + and d_moy=12 + and d_year=1998 + union all + select ss_ext_sales_price as ext_price, + ss_sold_date_sk as sold_date_sk, + ss_item_sk as sold_item_sk, + ss_sold_time_sk as time_sk + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + and d_moy=12 + and d_year=1998 + ) tmp,time_dim + where + sold_item_sk = i_item_sk + and i_manager_id=1 + and time_sk = t_time_sk + and (t_meal_time = 'breakfast' or t_meal_time = 'dinner') + group by i_brand, i_brand_id,t_hour,t_minute + order by ext_price desc, i_brand_id + ; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.71" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF0[d_date_sk->[ws_sold_date_sk],RF1[d_date_sk->[cs_sold_date_sk],RF2[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf72.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf72.groovy new file mode 100644 index 000000000000000..941ae4d3c744536 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf72.groovy @@ -0,0 +1,83 @@ +/* + * 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("ds_rf72") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_desc + ,w_warehouse_name + ,d1.d_week_seq + ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo + ,sum(case when p_promo_sk is not null then 1 else 0 end) promo + ,count(*) total_cnt +from catalog_sales +join inventory on (cs_item_sk = inv_item_sk) +join warehouse on (w_warehouse_sk=inv_warehouse_sk) +join item on (i_item_sk = cs_item_sk) +join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) +join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) +join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) +join date_dim d2 on (inv_date_sk = d2.d_date_sk) +join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) +left outer join promotion on (cs_promo_sk=p_promo_sk) +left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) +where d1.d_week_seq = d2.d_week_seq + and inv_quantity_on_hand < cs_quantity + and (d3.d_date > (d1.d_date + INTERVAL '5' DAY)) + and hd_buy_potential = '501-1000' + and d1.d_year = 2002 + and cd_marital_status = 'W' +group by i_item_desc,w_warehouse_name,d1.d_week_seq +order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.72" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF9[cs_item_sk->[cr_item_sk],RF10[cs_order_number->[cr_order_number],RF8[w_warehouse_sk->[inv_warehouse_sk],RF7[cs_item_sk->[i_item_sk],RF5[d_week_seq->[d_week_seq],RF6[d_date_sk->[inv_date_sk],RF4[cs_item_sk->[inv_item_sk],RF3[d_date_sk->[cs_ship_date_sk],RF2[cd_demo_sk->[cs_bill_cdemo_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[hd_demo_sk->[cs_bill_hdemo_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf73.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf73.groovy new file mode 100644 index 000000000000000..ae91db3b06aface --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf73.groovy @@ -0,0 +1,82 @@ +/* + * 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("ds_rf73") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_buy_potential = '501-1000' or + household_demographics.hd_buy_potential = 'Unknown') + and household_demographics.hd_vehicle_count > 0 + and case when household_demographics.hd_vehicle_count > 0 then + household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_county in ('Fairfield County','Walker County','Daviess County','Barrow County') + group by ss_ticket_number,ss_customer_sk) dj,customer + where ss_customer_sk = c_customer_sk + and cnt between 1 and 5 + order by cnt desc, c_last_name asc; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.73" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[ss_customer_sk->[c_customer_sk],RF2[s_store_sk->[ss_store_sk],RF1[hd_demo_sk->[ss_hdemo_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf74.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf74.groovy new file mode 100644 index 000000000000000..3d07c889d9565ed --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf74.groovy @@ -0,0 +1,115 @@ +/* + * 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("ds_rf74") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,stddev_samp(ss_net_paid) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (1999,1999+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,stddev_samp(ws_net_paid) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + and d_year in (1999,1999+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + ) + select + t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.year = 1999 + and t_s_secyear.year = 1999+1 + and t_w_firstyear.year = 1999 + and t_w_secyear.year = 1999+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + order by 2,1,3 +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.74" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[c_customer_sk->[ss_customer_sk],RF0[d_date_sk->[ss_sold_date_sk],RF3[c_customer_sk->[ws_bill_customer_sk],RF2[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query75.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf75.groovy similarity index 54% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query75.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf75.groovy index 6d580337f74e5ca..eb6fe5390fb6345 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query75.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf75.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -WITH all_sales AS ( +/* + * 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("ds_rf75") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + WITH all_sales AS ( SELECT d_year ,i_brand_id ,i_class_id @@ -70,3 +99,26 @@ WITH all_sales AS ( AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 ORDER BY sales_cnt_diff,sales_amt_diff limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.75" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[cs_order_number->[cr_order_number],RF3[cs_item_sk->[cr_item_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[i_item_sk->[cs_item_sk],RF6[ss_ticket_number->[sr_ticket_number],RF7[ss_item_sk->[sr_item_sk],RF5[d_date_sk->[ss_sold_date_sk],RF4[i_item_sk->[ss_item_sk],RF10[ws_order_number->[wr_order_number],RF11[ws_item_sk->[wr_item_sk],RF9[d_date_sk->[ws_sold_date_sk],RF8[i_item_sk->[ws_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf76.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf76.groovy new file mode 100644 index 000000000000000..e34cdeb7c32f4c3 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf76.groovy @@ -0,0 +1,78 @@ +/* + * 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("ds_rf76") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( + SELECT 'store' as channel, 'ss_hdemo_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_hdemo_sk IS NULL + AND ss_sold_date_sk=d_date_sk + AND ss_item_sk=i_item_sk + UNION ALL + SELECT 'web' as channel, 'ws_bill_addr_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_bill_addr_sk IS NULL + AND ws_sold_date_sk=d_date_sk + AND ws_item_sk=i_item_sk + UNION ALL + SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_warehouse_sk IS NULL + AND cs_sold_date_sk=d_date_sk + AND cs_item_sk=i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.76" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_sk->[ss_item_sk],RF3[d_date_sk->[ws_sold_date_sk],RF2[i_item_sk->[ws_item_sk],RF5[d_date_sk->[cs_sold_date_sk],RF4[i_item_sk->[cs_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query77.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf77.groovy similarity index 56% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query77.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf77.groovy index d2527a44b8863da..b7e8d8fca1c4d47 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query77.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf77.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -with ss as +/* + * 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("ds_rf77") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ss as (select s_store_sk, sum(ss_ext_sales_price) as sales, sum(ss_net_profit) as profit @@ -108,3 +137,26 @@ with ss as order by channel ,id limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.77" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[s_store_sk->[ss_store_sk],RF2[d_date_sk->[ss_sold_date_sk],RF1[s_store_sk->[sr_store_sk],RF0[d_date_sk->[sr_returned_date_sk],RF5[d_date_sk->[cs_sold_date_sk],RF4[d_date_sk->[cr_returned_date_sk],RF9[wp_web_page_sk->[ws_web_page_sk],RF8[d_date_sk->[ws_sold_date_sk],RF7[wp_web_page_sk->[wr_web_page_sk],RF6[d_date_sk->[wr_returned_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf78.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf78.groovy new file mode 100644 index 000000000000000..eece3c74cf1cc92 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf78.groovy @@ -0,0 +1,112 @@ +/* + * 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("ds_rf78") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ws as + (select d_year AS ws_sold_year, ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + from web_sales + left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk + join date_dim on ws_sold_date_sk = d_date_sk + where wr_order_number is null + group by d_year, ws_item_sk, ws_bill_customer_sk + ), +cs as + (select d_year AS cs_sold_year, cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + from catalog_sales + left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk + join date_dim on cs_sold_date_sk = d_date_sk + where cr_order_number is null + group by d_year, cs_item_sk, cs_bill_customer_sk + ), +ss as + (select d_year AS ss_sold_year, ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + from store_sales + left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk + join date_dim on ss_sold_date_sk = d_date_sk + where sr_ticket_number is null + group by d_year, ss_item_sk, ss_customer_sk + ) + select +ss_item_sk, +round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio, +ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price, +coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty, +coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost, +coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price +from ss +left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk) +left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk) +where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=2000 +order by + ss_item_sk, + ss_qty desc, ss_wc desc, ss_sp desc, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + ratio +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.78" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[d_date_sk->[ss_sold_date_sk],RF1[d_date_sk->[ws_sold_date_sk],RF0[d_date_sk->[cs_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf79.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf79.groovy new file mode 100644 index 000000000000000..bc8dd97eed0803d --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf79.groovy @@ -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. + */ + +suite("ds_rf79") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,store.s_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (household_demographics.hd_dep_count = 5 or household_demographics.hd_vehicle_count > 4) + and date_dim.d_dow = 1 + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_number_employees between 200 and 295 + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer + where ss_customer_sk = c_customer_sk + order by c_last_name,c_first_name,substr(s_city,1,30), profit +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.79" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[c_customer_sk->[ss_customer_sk],RF2[s_store_sk->[ss_store_sk],RF1[hd_demo_sk->[ss_hdemo_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query8.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf8.groovy similarity index 73% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query8.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf8.groovy index 9d17be9ea53dd5e..5b9da105d21928c 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query8.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf8.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -select s_store_name +/* + * 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("ds_rf8") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select s_store_name ,sum(ss_net_profit) from store_sales ,date_dim @@ -108,3 +137,26 @@ select s_store_name group by s_store_name order by s_store_name limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.8" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[s_store_sk->[ss_store_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[c_current_addr_sk->[ca_address_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query80.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf80.groovy similarity index 51% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query80.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf80.groovy index da8c3852422b734..6560db129c42668 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query80.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf80.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -with ssr as +/* + * 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("ds_rf80") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ssr as (select s_store_id as store_id, sum(ss_ext_sales_price) as sales, sum(coalesce(sr_return_amt, 0)) as returns, @@ -96,3 +125,26 @@ group by web_site_id) order by channel ,id limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.80" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF4[ss_item_sk->[sr_item_sk],RF5[ss_ticket_number->[sr_ticket_number],RF3[s_store_sk->[ss_store_sk],RF2[p_promo_sk->[ss_promo_sk],RF1[i_item_sk->[ss_item_sk],RF0[d_date_sk->[ss_sold_date_sk],RF10[cs_item_sk->[cr_item_sk],RF11[cs_order_number->[cr_order_number],RF9[cp_catalog_page_sk->[cs_catalog_page_sk],RF8[p_promo_sk->[cs_promo_sk],RF7[i_item_sk->[cs_item_sk],RF6[d_date_sk->[cs_sold_date_sk],RF16[ws_item_sk->[wr_item_sk],RF17[ws_order_number->[wr_order_number],RF15[web_site_sk->[ws_web_site_sk],RF14[p_promo_sk->[ws_promo_sk],RF13[i_item_sk->[ws_item_sk],RF12[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf81.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf81.groovy new file mode 100644 index 000000000000000..f6f856a277c68e5 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf81.groovy @@ -0,0 +1,85 @@ +/* + * 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("ds_rf81") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with customer_total_return as + (select cr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(cr_return_amt_inc_tax) as ctr_total_return + from catalog_returns + ,date_dim + ,customer_address + where cr_returned_date_sk = d_date_sk + and d_year =2002 + and cr_returning_addr_sk = ca_address_sk + group by cr_returning_customer_sk + ,ca_state ) + select c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'CA' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.81" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[ca_address_sk->[cr_returning_addr_sk],RF0[d_date_sk->[cr_returned_date_sk],RF3[c_customer_sk->[ctr_customer_sk],RF2[ca_address_sk->[c_current_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf82.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf82.groovy new file mode 100644 index 000000000000000..0502e0ce2b9cdfd --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf82.groovy @@ -0,0 +1,71 @@ +/* + * 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("ds_rf82") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, store_sales + where i_current_price between 17 and 17+30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('1999-07-09' as date) and (cast('1999-07-09' as date) + interval 60 day) + and i_manufact_id in (639,169,138,339) + and inv_quantity_on_hand between 100 and 500 + and ss_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.82" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[i_item_sk->[ss_item_sk],RF1[d_date_sk->[inv_date_sk],RF0[i_item_sk->[inv_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf83.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf83.groovy new file mode 100644 index 000000000000000..95dc69699faf3fb --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf83.groovy @@ -0,0 +1,121 @@ +/* + * 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("ds_rf83") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with sr_items as + (select i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + from store_returns, + item, + date_dim + where sr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2001-06-06','2001-09-02','2001-11-11'))) + and sr_returned_date_sk = d_date_sk + group by i_item_id), + cr_items as + (select i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + from catalog_returns, + item, + date_dim + where cr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2001-06-06','2001-09-02','2001-11-11'))) + and cr_returned_date_sk = d_date_sk + group by i_item_id), + wr_items as + (select i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + from web_returns, + item, + date_dim + where wr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2001-06-06','2001-09-02','2001-11-11'))) + and wr_returned_date_sk = d_date_sk + group by i_item_id) + select sr_items.item_id + ,sr_item_qty + ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev + ,cr_item_qty + ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev + ,wr_item_qty + ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev + ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average + from sr_items + ,cr_items + ,wr_items + where sr_items.item_id=cr_items.item_id + and sr_items.item_id=wr_items.item_id + order by sr_items.item_id + ,sr_item_qty + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.83" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF13[item_id->[i_item_id],RF12[d_date->[d_date],RF11[d_date_sk->[cr_returned_date_sk],RF10[i_item_sk->[cr_item_sk],RF9[d_week_seq->[d_week_seq],RF8[item_id->[i_item_id],RF7[d_date->[d_date],RF6[d_date_sk->[sr_returned_date_sk],RF5[i_item_sk->[sr_item_sk],RF4[d_week_seq->[d_week_seq],RF3[d_date->[d_date],RF2[d_date_sk->[wr_returned_date_sk],RF1[i_item_sk->[wr_item_sk],RF0[d_week_seq->[d_week_seq]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf84.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf84.groovy new file mode 100644 index 000000000000000..9517c3789d26032 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf84.groovy @@ -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. + */ + +suite("ds_rf84") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select c_customer_id as customer_id + , concat(concat(coalesce(c_last_name,''), ','), coalesce(c_first_name,'')) as customername + from customer + ,customer_address + ,customer_demographics + ,household_demographics + ,income_band + ,store_returns + where ca_city = 'Oakwood' + and c_current_addr_sk = ca_address_sk + and ib_lower_bound >= 5806 + and ib_upper_bound <= 5806 + 50000 + and ib_income_band_sk = hd_income_band_sk + and cd_demo_sk = c_current_cdemo_sk + and hd_demo_sk = c_current_hdemo_sk + and sr_cdemo_sk = cd_demo_sk + order by c_customer_id + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.84" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF4[cd_demo_sk->[sr_cdemo_sk],RF3[c_current_cdemo_sk->[cd_demo_sk],RF2[hd_demo_sk->[c_current_hdemo_sk],RF1[ca_address_sk->[c_current_addr_sk],RF0[ib_income_band_sk->[hd_income_band_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf85.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf85.groovy new file mode 100644 index 000000000000000..a7e014cd45ae5b2 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf85.groovy @@ -0,0 +1,138 @@ +/* + * 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("ds_rf85") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) + from web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason + where ws_web_page_sk = wp_web_page_sk + and ws_item_sk = wr_item_sk + and ws_order_number = wr_order_number + and ws_sold_date_sk = d_date_sk and d_year = 2000 + and cd1.cd_demo_sk = wr_refunded_cdemo_sk + and cd2.cd_demo_sk = wr_returning_cdemo_sk + and ca_address_sk = wr_refunded_addr_sk + and r_reason_sk = wr_reason_sk + and + ( + ( + cd1.cd_marital_status = 'M' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = '4 yr Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 100.00 and 150.00 + ) + or + ( + cd1.cd_marital_status = 'S' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Secondary' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 50.00 and 100.00 + ) + or + ( + cd1.cd_marital_status = 'W' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Advanced Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ca_country = 'United States' + and + ca_state in ('FL', 'TX', 'DE') + and ws_net_profit between 100 and 200 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('IN', 'ND', 'ID') + and ws_net_profit between 150 and 300 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('MT', 'IL', 'OH') + and ws_net_profit between 50 and 250 + ) + ) +group by r_reason_desc +order by substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.85" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF7[cd_marital_status->[cd_marital_status],RF8[cd_education_status->[cd_education_status],RF9[wr_returning_cdemo_sk->[cd_demo_sk],RF6[ws_web_page_sk->[wp_web_page_sk],RF5[wr_refunded_cdemo_sk->[cd_demo_sk],RF4[wr_reason_sk->[r_reason_sk],RF3[wr_refunded_addr_sk->[ca_address_sk],RF1[ws_item_sk->[wr_item_sk],RF2[ws_order_number->[wr_order_number],RF0[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf86.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf86.groovy new file mode 100644 index 000000000000000..f28085ad01de6fd --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf86.groovy @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +suite("ds_rf86") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + sum(ws_net_paid) as total_sum + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ws_net_paid) desc) as rank_within_parent + from + web_sales + ,date_dim d1 + ,item + where + d1.d_month_seq between 1224 and 1224+11 + and d1.d_date_sk = ws_sold_date_sk + and i_item_sk = ws_item_sk + group by rollup(i_category,i_class) + order by + lochierarchy desc, + case when lochierarchy = 0 then i_category end, + rank_within_parent + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.86" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[i_item_sk->[ws_item_sk],RF0[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf87.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf87.groovy new file mode 100644 index 000000000000000..5eb74154b772ccf --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf87.groovy @@ -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. + */ + +suite("ds_rf87") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select count(*) +from ((select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1184 and 1184+11) + except + (select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1184 and 1184+11) + except + (select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1184 and 1184+11) +) cool_cust +; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.87" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[c_customer_sk->[ss_customer_sk],RF0[d_date_sk->[ss_sold_date_sk],RF3[c_customer_sk->[cs_bill_customer_sk],RF2[d_date_sk->[cs_sold_date_sk],RF5[c_customer_sk->[ws_bill_customer_sk],RF4[d_date_sk->[ws_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query88.sql b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf88.groovy similarity index 64% rename from regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query88.sql rename to regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf88.groovy index 7b823f9fce6a6d5..1cd5efbf805a2ac 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/query88.sql +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf88.groovy @@ -1,9 +1,38 @@ -explain shape plan - - - - -select * +/* + * 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("ds_rf88") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select * from (select count(*) h8_30_to_9 from store_sales, household_demographics , time_dim, store @@ -94,3 +123,26 @@ from (household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2)) and store.s_store_name = 'ese') s8 ; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.88" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF23[hd_demo_sk->[ss_hdemo_sk],RF22[s_store_sk->[ss_store_sk],RF21[t_time_sk->[ss_sold_time_sk],RF20[hd_demo_sk->[ss_hdemo_sk],RF19[s_store_sk->[ss_store_sk],RF18[t_time_sk->[ss_sold_time_sk],RF17[hd_demo_sk->[ss_hdemo_sk],RF16[s_store_sk->[ss_store_sk],RF15[t_time_sk->[ss_sold_time_sk],RF14[hd_demo_sk->[ss_hdemo_sk],RF13[s_store_sk->[ss_store_sk],RF12[t_time_sk->[ss_sold_time_sk],RF11[hd_demo_sk->[ss_hdemo_sk],RF10[s_store_sk->[ss_store_sk],RF9[t_time_sk->[ss_sold_time_sk],RF8[hd_demo_sk->[ss_hdemo_sk],RF7[s_store_sk->[ss_store_sk],RF6[t_time_sk->[ss_sold_time_sk],RF5[hd_demo_sk->[ss_hdemo_sk],RF4[s_store_sk->[ss_store_sk],RF3[t_time_sk->[ss_sold_time_sk],RF2[hd_demo_sk->[ss_hdemo_sk],RF1[s_store_sk->[ss_store_sk],RF0[t_time_sk->[ss_sold_time_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf89.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf89.groovy new file mode 100644 index 000000000000000..14d5a65401492cf --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf89.groovy @@ -0,0 +1,82 @@ +/* + * 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("ds_rf89") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select * +from( +select i_category, i_class, i_brand, + s_store_name, s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + d_year in (1999) and + ((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') + )) +group by i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 +order by sum_sales - avg_monthly_sales, s_store_name +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.89" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[s_store_sk->[ss_store_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[i_item_sk->[ss_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf9.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf9.groovy new file mode 100644 index 000000000000000..cc7e327057668cd --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf9.groovy @@ -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. + */ + +suite("ds_rf9") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select case when (select count(*) + from store_sales + where ss_quantity between 1 and 20) > 2972190 + then (select avg(ss_ext_sales_price) + from store_sales + where ss_quantity between 1 and 20) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 1 and 20) end bucket1 , + case when (select count(*) + from store_sales + where ss_quantity between 21 and 40) > 4505785 + then (select avg(ss_ext_sales_price) + from store_sales + where ss_quantity between 21 and 40) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 21 and 40) end bucket2, + case when (select count(*) + from store_sales + where ss_quantity between 41 and 60) > 1575726 + then (select avg(ss_ext_sales_price) + from store_sales + where ss_quantity between 41 and 60) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 41 and 60) end bucket3, + case when (select count(*) + from store_sales + where ss_quantity between 61 and 80) > 3188917 + then (select avg(ss_ext_sales_price) + from store_sales + where ss_quantity between 61 and 80) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 61 and 80) end bucket4, + case when (select count(*) + from store_sales + where ss_quantity between 81 and 100) > 3525216 + then (select avg(ss_ext_sales_price) + from store_sales + where ss_quantity between 81 and 100) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 81 and 100) end bucket5 +from reason +where r_reason_sk = 1 +; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.9" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf90.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf90.groovy new file mode 100644 index 000000000000000..31711a590ec8580 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf90.groovy @@ -0,0 +1,76 @@ +/* + * 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("ds_rf90") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio + from ( select count(*) amc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 10 and 10+1 + and household_demographics.hd_dep_count = 2 + and web_page.wp_char_count between 5000 and 5200) at, + ( select count(*) pmc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 16 and 16+1 + and household_demographics.hd_dep_count = 2 + and web_page.wp_char_count between 5000 and 5200) pt + order by am_pm_ratio + limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.90" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[hd_demo_sk->[ws_ship_hdemo_sk],RF4[t_time_sk->[ws_sold_time_sk],RF3[wp_web_page_sk->[ws_web_page_sk],RF2[hd_demo_sk->[ws_ship_hdemo_sk],RF1[t_time_sk->[ws_sold_time_sk],RF0[wp_web_page_sk->[ws_web_page_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf91.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf91.groovy new file mode 100644 index 000000000000000..7b03ffcd2b2e14d --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf91.groovy @@ -0,0 +1,85 @@ +/* + * 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("ds_rf91") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +from + call_center, + catalog_returns, + date_dim, + customer, + customer_address, + customer_demographics, + household_demographics +where + cr_call_center_sk = cc_call_center_sk +and cr_returned_date_sk = d_date_sk +and cr_returning_customer_sk= c_customer_sk +and cd_demo_sk = c_current_cdemo_sk +and hd_demo_sk = c_current_hdemo_sk +and ca_address_sk = c_current_addr_sk +and d_year = 2001 +and d_moy = 11 +and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') + or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) +and hd_buy_potential like '1001-5000%' +and ca_gmt_offset = -6 +group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status +order by sum(cr_net_loss) desc; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.91" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF5[c_current_addr_sk->[ca_address_sk],RF4[c_current_cdemo_sk->[cd_demo_sk],RF3[hd_demo_sk->[c_current_hdemo_sk],RF2[cr_returning_customer_sk->[c_customer_sk],RF1[d_date_sk->[cr_returned_date_sk],RF0[cc_call_center_sk->[cr_call_center_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf92.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf92.groovy new file mode 100644 index 000000000000000..4f3e0e548bce283 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf92.groovy @@ -0,0 +1,84 @@ +/* + * 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("ds_rf92") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + sum(ws_ext_discount_amt) as "Excess Discount Amount" +from + web_sales + ,item + ,date_dim +where +i_manufact_id = 320 +and i_item_sk = ws_item_sk +and d_date between '2002-02-26' and + (cast('2002-02-26' as date) + interval 90 day) +and d_date_sk = ws_sold_date_sk +and ws_ext_discount_amt + > ( + SELECT + 1.3 * avg(ws_ext_discount_amt) + FROM + web_sales + ,date_dim + WHERE + ws_item_sk = i_item_sk + and d_date between '2002-02-26' and + (cast('2002-02-26' as date) + interval 90 day) + and d_date_sk = ws_sold_date_sk + ) +order by sum(ws_ext_discount_amt) +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.92" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[ws_sold_date_sk],RF0[i_item_sk->[ws_item_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf93.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf93.groovy new file mode 100644 index 000000000000000..160a50b795c31b0 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf93.groovy @@ -0,0 +1,72 @@ +/* + * 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("ds_rf93") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select ss_customer_sk + ,sum(act_sales) sumsales + from (select ss_item_sk + ,ss_ticket_number + ,ss_customer_sk + ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price + else (ss_quantity*ss_sales_price) end act_sales + from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk + and sr_ticket_number = ss_ticket_number) + ,reason + where sr_reason_sk = r_reason_sk + and r_reason_desc = 'duplicate purchase') t + group by ss_customer_sk + order by sumsales, ss_customer_sk +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.93" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[sr_item_sk->[ss_item_sk],RF2[sr_ticket_number->[ss_ticket_number],RF0[r_reason_sk->[sr_reason_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf94.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf94.groovy new file mode 100644 index 000000000000000..710098a31520f59 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf94.groovy @@ -0,0 +1,83 @@ +/* + * 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("ds_rf94") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + count(distinct ws_order_number) as "order count" + ,sum(ws_ext_ship_cost) as "total shipping cost" + ,sum(ws_net_profit) as "total net profit" +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '2000-2-01' and + (cast('2000-2-01' as date) + interval 60 day) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'OK' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and exists (select * + from web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +and not exists(select * + from web_returns wr1 + where ws1.ws_order_number = wr1.wr_order_number) +order by count(distinct ws_order_number) +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.94" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF4[d_date_sk->[ws_ship_date_sk],RF3[ws_order_number->[ws_order_number],RF2[ws_order_number->[wr_order_number],RF1[web_site_sk->[ws_web_site_sk],RF0[ca_address_sk->[ws_ship_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf95.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf95.groovy new file mode 100644 index 000000000000000..3785a77a131baa2 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf95.groovy @@ -0,0 +1,86 @@ +/* + * 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("ds_rf95") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ws_wh as +(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2 + from web_sales ws1,web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + select + count(distinct ws_order_number) as "order count" + ,sum(ws_ext_ship_cost) as "total shipping cost" + ,sum(ws_net_profit) as "total net profit" +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '1999-2-01' and + (cast('1999-2-01' as date) + interval 60 day) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'NC' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and ws1.ws_order_number in (select ws_order_number + from ws_wh) +and ws1.ws_order_number in (select wr_order_number + from web_returns,ws_wh + where wr_order_number = ws_wh.ws_order_number) +order by count(distinct ws_order_number) +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.95" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF0[ws_order_number->[ws_order_number],RF7[ws_ship_date_sk->[d_date_sk],RF4[wr_order_number->[ws_order_number],RF5[wr_order_number->[ws_order_number, ws_order_number],RF3[ws_order_number->[ws_order_number],RF6[ws_order_number->[ws_order_number, ws_order_number],RF2[web_site_sk->[ws_web_site_sk],RF1[ca_address_sk->[ws_ship_addr_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf96.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf96.groovy new file mode 100644 index 000000000000000..9057e7963082f90 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf96.groovy @@ -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. + */ + +suite("ds_rf96") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select count(*) +from store_sales + ,household_demographics + ,time_dim, store +where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and household_demographics.hd_dep_count = 3 + and store.s_store_name = 'ese' +order by count(*) +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.96" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF2[s_store_sk->[ss_store_sk],RF1[hd_demo_sk->[ss_hdemo_sk],RF0[t_time_sk->[ss_sold_time_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf97.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf97.groovy new file mode 100644 index 000000000000000..38dadc718972fbb --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf97.groovy @@ -0,0 +1,79 @@ +/* + * 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("ds_rf97") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + with ssci as ( +select ss_customer_sk customer_sk + ,ss_item_sk item_sk +from store_sales,date_dim +where ss_sold_date_sk = d_date_sk + and d_month_seq between 1214 and 1214 + 11 +group by ss_customer_sk + ,ss_item_sk), +csci as( + select cs_bill_customer_sk customer_sk + ,cs_item_sk item_sk +from catalog_sales,date_dim +where cs_sold_date_sk = d_date_sk + and d_month_seq between 1214 and 1214 + 11 +group by cs_bill_customer_sk + ,cs_item_sk) + select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only + ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only + ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog +from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk + and ssci.item_sk = csci.item_sk) +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.97" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[d_date_sk->[ss_sold_date_sk],RF0[d_date_sk->[cs_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf98.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf98.groovy new file mode 100644 index 000000000000000..e9dac0f59e14b09 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf98.groovy @@ -0,0 +1,87 @@ +/* + * 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("ds_rf98") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ss_ext_sales_price) as itemrevenue + ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over + (partition by i_class) as revenueratio +from + store_sales + ,item + ,date_dim +where + ss_item_sk = i_item_sk + and i_category in ('Sports', 'Music', 'Shoes') + and ss_sold_date_sk = d_date_sk + and d_date between cast('2002-05-20' as date) + and (cast('2002-05-20' as date) + interval 30 day) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.98" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF1[i_item_sk->[ss_item_sk],RF0[d_date_sk->[ss_sold_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf99.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf99.groovy new file mode 100644 index 000000000000000..032b0dec3b3e4f8 --- /dev/null +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf99.groovy @@ -0,0 +1,89 @@ +/* + * 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("ds_rf99") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + sql 'set enable_pipeline_engine=true' + String stmt = ''' + explain physical plan + select + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as "30 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and + (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as "31-60 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and + (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as "61-90 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and + (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as "91-120 days" + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as ">120 days" +from + catalog_sales + ,warehouse + ,ship_mode + ,call_center + ,date_dim +where + d_month_seq between 1224 and 1224 + 11 +and cs_ship_date_sk = d_date_sk +and cs_warehouse_sk = w_warehouse_sk +and cs_ship_mode_sk = sm_ship_mode_sk +and cs_call_center_sk = cc_call_center_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +limit 100; + + ''' + String plan = sql "${stmt}" + println plan + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + + // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.99" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + + assertEquals("RF3[w_warehouse_sk->[cs_warehouse_sk],RF2[sm_ship_mode_sk->[cs_ship_mode_sk],RF1[cc_call_center_sk->[cs_call_center_sk],RF0[d_date_sk->[cs_ship_date_sk]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/gen_rf.py b/regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/gen_rf.py new file mode 100644 index 000000000000000..4d9ff10944c73e3 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/gen_rf.py @@ -0,0 +1,29 @@ +# // 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. +if __name__ == '__main__': + with open('rf.tmpl', 'r') as f: + tmpl = f.read() + for i in range(1,23): + with open('../../../../tools/tpch-tools/queries/q'+str(i)+'.sql', 'r') as fi: + casei = tmpl.replace('{--}', str(i)) + casei = casei.replace('{query}', fi.read()) + # with open('../rf/h_rf'+str(i)+'.groovy', 'w') as out: + # out.write(casei) + with open('rf/rf.'+str(i), 'r') as rf_file: + casei = casei.replace('{rfs}', rf_file.read()) + with open('../rf/h_rf'+str(i)+'.groovy', 'w') as out: + out.write(casei) \ No newline at end of file diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf.tmpl b/regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf.tmpl new file mode 100644 index 000000000000000..fb92ddc5cfae0a8 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf.tmpl @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +suite("h_rf{--}") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + {query} + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.{--}" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("{rfs}", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf1.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf1.groovy new file mode 100644 index 000000000000000..926d36c3ee30094 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf1.groovy @@ -0,0 +1,92 @@ +/* + * 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("h_rf1") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.1" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf10.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf10.groovy new file mode 100644 index 000000000000000..254cbae94df4774 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf10.groovy @@ -0,0 +1,104 @@ +/* + * 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("h_rf10") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.10" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF2[o_orderkey->[l_orderkey],RF1[n_nationkey->[c_nationkey],RF0[o_custkey->[c_custkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf11.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf11.groovy new file mode 100644 index 000000000000000..d92751767528013 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf11.groovy @@ -0,0 +1,97 @@ +/* + * 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("h_rf11") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.11" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF3[s_suppkey->[ps_suppkey],RF2[n_nationkey->[s_nationkey],RF1[s_suppkey->[ps_suppkey],RF0[n_nationkey->[s_nationkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf12.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf12.groovy new file mode 100644 index 000000000000000..f61e03369721327 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf12.groovy @@ -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. + */ + +suite("h_rf12") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.12" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF0[l_orderkey->[o_orderkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf13.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf13.groovy new file mode 100644 index 000000000000000..a3960fb40d6d269 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf13.groovy @@ -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. + */ + +suite("h_rf13") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.13" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF0[c_custkey->[o_custkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf14.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf14.groovy new file mode 100644 index 000000000000000..92f5f3e92289d99 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf14.groovy @@ -0,0 +1,84 @@ +/* + * 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("h_rf14") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.14" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF0[l_partkey->[p_partkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf15.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf15.groovy new file mode 100644 index 000000000000000..80b5f0d174686d7 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf15.groovy @@ -0,0 +1,90 @@ +/* + * 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("h_rf15") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.15" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF0[supplier_no->[s_suppkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf16.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf16.groovy new file mode 100644 index 000000000000000..ab8e6b374233653 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf16.groovy @@ -0,0 +1,101 @@ +/* + * 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("h_rf16") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.16" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF0[p_partkey->[ps_partkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf17.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf17.groovy new file mode 100644 index 000000000000000..4ba22b7f2778e58 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf17.groovy @@ -0,0 +1,90 @@ +/* + * 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("h_rf17") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +-- Modified + +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 + ); + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.17" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF0[p_partkey->[l_partkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf18.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf18.groovy new file mode 100644 index 000000000000000..7b6efe186c7fd38 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf18.groovy @@ -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. + */ + +suite("h_rf18") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.18" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF2[o_orderkey->[l_orderkey],RF1[o_custkey->[c_custkey],RF0[l_orderkey->[o_orderkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf19.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf19.groovy new file mode 100644 index 000000000000000..8c76d12e0332940 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf19.groovy @@ -0,0 +1,106 @@ +/* + * 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("h_rf19") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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' + ); + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.19" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF0[p_partkey->[l_partkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf2.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf2.groovy new file mode 100644 index 000000000000000..9f06ce145a66edd --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf2.groovy @@ -0,0 +1,114 @@ +/* + * 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("h_rf2") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.2" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF3[ps_suppkey->[s_suppkey],RF2[n_nationkey->[s_nationkey],RF1[r_regionkey->[n_regionkey],RF0[p_partkey->[ps_partkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf20.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf20.groovy new file mode 100644 index 000000000000000..2ab6eef342edf3d --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf20.groovy @@ -0,0 +1,108 @@ +/* + * 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("h_rf20") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.20" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF4[s_suppkey->[ps_suppkey],RF2[ps_partkey->[l_partkey],RF3[ps_suppkey->[l_suppkey],RF1[p_partkey->[ps_partkey],RF0[n_nationkey->[s_nationkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf21.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf21.groovy new file mode 100644 index 000000000000000..4abc1c991d4f1c5 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf21.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("h_rf21") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.21" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF4[l_orderkey->[o_orderkey],RF3[l_orderkey->[l_orderkey],RF2[l_orderkey->[l_orderkey],RF1[s_suppkey->[l_suppkey],RF0[n_nationkey->[s_nationkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf22.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf22.groovy new file mode 100644 index 000000000000000..71deeb278b24f37 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf22.groovy @@ -0,0 +1,108 @@ +/* + * 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("h_rf22") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.22" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF0[c_custkey->[o_custkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf3.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf3.groovy new file mode 100644 index 000000000000000..ba2f84a955dba00 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf3.groovy @@ -0,0 +1,94 @@ +/* + * 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("h_rf3") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.3" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF1[o_orderkey->[l_orderkey],RF0[c_custkey->[o_custkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf4.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf4.groovy new file mode 100644 index 000000000000000..da433b83b5eea43 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf4.groovy @@ -0,0 +1,92 @@ +/* + * 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("h_rf4") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.4" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF0[o_orderkey->[l_orderkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf5.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf5.groovy new file mode 100644 index 000000000000000..36735d0a4c6fe3f --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf5.groovy @@ -0,0 +1,95 @@ +/* + * 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("h_rf5") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.5" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF4[c_nationkey->[s_nationkey],RF5[c_custkey->[o_custkey],RF3[o_orderkey->[l_orderkey],RF2[s_suppkey->[l_suppkey],RF1[n_nationkey->[s_nationkey],RF0[r_regionkey->[n_regionkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf6.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf6.groovy new file mode 100644 index 000000000000000..8b4c0ee4d4561b1 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf6.groovy @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +suite("h_rf6") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.6" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf7.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf7.groovy new file mode 100644 index 000000000000000..3651f6e780e5ed4 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf7.groovy @@ -0,0 +1,109 @@ +/* + * 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("h_rf7") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.7" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF3[n_nationkey->[c_nationkey],RF4[o_custkey->[c_custkey],RF2[l_orderkey->[o_orderkey],RF1[s_suppkey->[l_suppkey],RF0[n_nationkey->[s_nationkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf8.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf8.groovy new file mode 100644 index 000000000000000..2524227d46d4941 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf8.groovy @@ -0,0 +1,108 @@ +/* + * 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("h_rf8") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.8" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF6[n_nationkey->[s_nationkey],RF5[l_suppkey->[s_suppkey],RF4[r_regionkey->[n_regionkey],RF3[n_nationkey->[c_nationkey],RF2[o_custkey->[c_custkey],RF1[l_orderkey->[o_orderkey],RF0[p_partkey->[l_partkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf9.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf9.groovy new file mode 100644 index 000000000000000..4595a0de0b43c38 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf9.groovy @@ -0,0 +1,103 @@ +/* + * 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("h_rf9") { + 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 be_number_for_test=3' + sql 'set parallel_fragment_exec_instance_num=8; ' + sql 'set parallel_pipeline_task_num=8; ' + sql 'set forbid_unknown_col_stats=true' + sql 'set broadcast_row_count_limit = 30000000' + sql 'set enable_nereids_timeout = false' + + String stmt = ''' + explain physical plan + -- 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. + +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; + + ''' + String plan = sql "${stmt}" + def getRuntimeFilters = { plantree -> + { + def lst = [] + plantree.eachMatch("RF\\d+\\[[^#]+#\\d+->\\[[^\\]]+\\]") { + ch -> + { + lst.add(ch.replaceAll("#\\d+", '')) + } + } + return lst.join(',') + } + } + // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.9" + // File file = new File(outFile) + // file.write(getRuntimeFilters(plan)) + assertEquals("RF4[ps_suppkey->[l_suppkey],RF5[ps_partkey->[l_partkey],RF3[l_orderkey->[o_orderkey],RF2[s_suppkey->[l_suppkey],RF1[p_partkey->[l_partkey],RF0[n_nationkey->[s_nationkey]", getRuntimeFilters(plan)) +} diff --git a/regression-test/suites/performance_p0/test_streamload_perfomance.groovy b/regression-test/suites/performance_p0/test_streamload_perfomance.groovy index aabcc6e9a6020bb..46f2341f8066093 100644 --- a/regression-test/suites/performance_p0/test_streamload_perfomance.groovy +++ b/regression-test/suites/performance_p0/test_streamload_perfomance.groovy @@ -44,7 +44,7 @@ suite("test_streamload_perfomance") { try_sql "DROP TABLE IF EXISTS ${tableName}" } - // test stream load with sql performance + // test http_stream performance try { sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( @@ -57,18 +57,17 @@ suite("test_streamload_perfomance") { ) """ - def rowCount = 10000 + def rowCount = 200 def rowIt = java.util.stream.LongStream.range(0, rowCount) .mapToObj({i -> [i, "a_" + i]}) .iterator() - // streamLoad { - // set 'sql', 'insert into regression_test_performance_p0.test_streamload_performance1 select * from stream("format"="csv", "column_separator"="\t")' - // table tableName - // set 'version', '1' - // time 100000 - // inputIterator rowIt - // } + streamLoad { + set 'version', '1' + set 'sql', 'insert into regression_test_performance_p0.test_streamload_performance1 select * from http_stream("format"="csv", "column_separator"="\t")' + time 100000 + inputIterator rowIt + } } finally { try_sql "DROP TABLE IF EXISTS ${tableName}" } diff --git a/regression-test/suites/query_p0/aggregate/bitmap_agg.groovy b/regression-test/suites/query_p0/aggregate/bitmap_agg.groovy index edfc246df042779..7ee79c618622151 100644 --- a/regression-test/suites/query_p0/aggregate/bitmap_agg.groovy +++ b/regression-test/suites/query_p0/aggregate/bitmap_agg.groovy @@ -69,4 +69,54 @@ suite("bitmap_agg") { """ sql "DROP TABLE IF EXISTS `test_bitmap_agg`;" + sql "DROP TABLE IF EXISTS `test_bitmap_agg_nation`;" + sql """ + CREATE TABLE `test_bitmap_agg_nation` ( + `N_NATIONKEY` int(11) NOT NULL, + `N_NAME` char(25) NOT NULL, + `N_REGIONKEY` int(11) NOT NULL, + `N_COMMENT` varchar(152) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`N_NATIONKEY`, `N_NAME`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`N_NATIONKEY`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + INSERT INTO `test_bitmap_agg_nation` VALUES (0,'ALGERIA',0,' haggle. carefully final deposits detect slyly agai'), + (1,'ARGENTINA',1,'al foxes promise slyly according to the regular accounts. bold requests alon'), + (3,'CANADA',1,'eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold'), + (8,'INDIA',2,'ss excuses cajole slyly across the packages. deposits print aroun'), + (13,'JORDAN',4,'ic deposits are blithely about the carefully regular pa'), + (17,'PERU',1,'platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun'), + (22,'RUSSIA',3,' requests against the platelets use never according to the quickly regular pint'), + (2,'BRAZIL',1,'y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special '), + (9,'INDONESIA',2,' slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull'), + (10,'IRAN',4,'efully alongside of the slyly final dependencies. '), + (11,'IRAQ',4,'nic deposits boost atop the quickly final requests? quickly regula'), + (12,'JAPAN',2,'ously. final, express gifts cajole a'), + (15,'MOROCCO',0,'rns. blithely bold courts among the closely regular packages use furiously bold platelets?'), + (18,'CHINA',2,'c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos'), + (21,'VIETNAM',2,'hely enticingly express accounts. even, final '), + (24,'UNITED STATES',1,'y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be'), + (14,'KENYA',0,' pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t'), + (16,'MOZAMBIQUE',0,'s. ironic, unusual asymptotes wake blithely r'), + (19,'ROMANIA',3,'ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account'), + (20,'SAUDI ARABIA',4,'ts. silent requests haggle. closely express packages sleep across the blithely'), + (23,'UNITED KINGDOM',3,'eans boost carefully special requests. accounts are. carefull'), + (4,'EGYPT',4,'y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d'), + (5,'ETHIOPIA',0,'ven packages wake quickly. regu'),(6,'FRANCE',3,'refully final requests. regular, ironi'), + (7,'GERMANY',3,'l platelets. regular accounts x-ray: unusual, regular acco'); + """ + + qt_sql3 """ + select count(`n_nationkey`), count(distinct `n_nationkey`), bitmap_count(bitmap_agg(`n_nationkey`)) from `test_bitmap_agg_nation`; + """ + qt_sql4 """ + select count(`n_nationkey`), bitmap_count(bitmap_agg(`n_nationkey`)) from `test_bitmap_agg_nation` group by `n_regionkey`; + """ + sql "DROP TABLE IF EXISTS `test_bitmap_agg_nation`;" } diff --git a/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy b/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy index bad3af4c69e3590..cbe09ec527ffbcc 100644 --- a/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy +++ b/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy @@ -58,8 +58,11 @@ suite("test_partitioned_hash_join", "query,p0") { assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) } } + sql """ set enable_profile = 1; """ sql "insert into test_partitioned_hash_join_l values (100, 1100), (0, 10), (1, 110), (255, 2550)"; + sql """ sync """ + qt_partitioned_hash_join1 """ select /*+SET_VAR(disable_join_reorder=true,experimental_enable_pipeline_engine=false, parallel_fragment_exec_instance_num=1, partitioned_hash_join_rows_threshold = 1)*/ diff --git a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions_by_literal.groovy b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions_by_literal.groovy index f2ac1da429083f1..f35549019e5aaca 100644 --- a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions_by_literal.groovy +++ b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions_by_literal.groovy @@ -17,6 +17,7 @@ suite("test_array_functions_by_literal") { // array_nested function + sql """ set enable_nereids_planner = false; """ qt_sql "select a from (select array(1, 1, 2, 2, 2, 2) as a) t" // array with decimal and other types diff --git a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_floor_ceil.groovy b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_floor_ceil.groovy new file mode 100644 index 000000000000000..5d374c8fbab5b73 --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_floor_ceil.groovy @@ -0,0 +1,36 @@ +// 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_date_floor_ceil") { + sql "set enable_nereids_planner=true;" + sql "set enable_fallback_to_original_planner=false;" + sql "set enable_fold_constant_by_be=false;" + + qt_sql1 """select date_floor("2023-07-14 10:51:11",interval 5 second); """ + qt_sql2 """select date_floor("2023-07-14 10:51:00",interval 5 minute); """ + qt_sql3 """select date_floor("2023-07-14 10:51:00",interval 5 hour); """ + qt_sql4 """select date_floor("2023-07-14 10:51:00",interval 5 day); """ + qt_sql5 """select date_floor("2023-07-14 10:51:00",interval 5 month); """ + qt_sql6 """select date_floor("2023-07-14 10:51:00",interval 5 year); """ + + qt_sql7 """select date_ceil("2023-07-14 10:51:11",interval 5 second); """ + qt_sql8 """select date_ceil("2023-07-14 10:51:00",interval 5 minute); """ + qt_sql9 """select date_ceil("2023-07-14 10:51:00",interval 5 hour); """ + qt_sql10 """select date_ceil("2023-07-14 10:51:00",interval 5 day); """ + qt_sql11 """select date_ceil("2023-07-14 10:51:00",interval 5 month); """ + qt_sql12 """select date_ceil("2023-07-14 10:51:00",interval 5 year); """ +} \ No newline at end of file diff --git a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function_regexp.groovy b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function_regexp.groovy index cb80939adf38fbe..e3e04a5456f50d5 100644 --- a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function_regexp.groovy +++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function_regexp.groovy @@ -56,6 +56,8 @@ suite("test_string_function_regexp") { qt_sql "SELECT regexp_extract_all('http://a.m.baidu.com/i41915i73660.htm', 'i([0-9]+)');" qt_sql "SELECT regexp_extract_all('abc=111, def=222, ghi=333', '(\"[^\"]+\"|\\\\w+)=(\"[^\"]+\"|\\\\w+)');" qt_sql "select regexp_extract_all('xxfs','f');" + qt_sql "select regexp_extract_all('asdfg', '(z|x|c|)');" + qt_sql "select regexp_extract_all('abcdfesscca', '(ab|c|)');" qt_sql "SELECT regexp_replace('a b c', \" \", \"-\");" qt_sql "SELECT regexp_replace('a b c','(b)','<\\\\1>');" diff --git a/regression-test/suites/query_p0/sql_functions/table_function/explode.groovy b/regression-test/suites/query_p0/sql_functions/table_function/explode.groovy index 7118ebf43bb07fc..fabb847189ae055 100644 --- a/regression-test/suites/query_p0/sql_functions/table_function/explode.groovy +++ b/regression-test/suites/query_p0/sql_functions/table_function/explode.groovy @@ -49,4 +49,25 @@ suite("explode") { qt_test1 """select e1 from (select k1 from d_table) as t lateral view explode_numbers(5) tmp1 as e1;""" qt_test2 """select e1 from (select k1 from d_table) as t lateral view explode_numbers(5) tmp1 as e1 where e1=k1;""" qt_test3 """select e1,k1 from (select k1 from d_table) as t lateral view explode_numbers(5) tmp1 as e1;""" + + sql """ DROP TABLE IF EXISTS baseall_explode_numbers; """ + sql """ + CREATE TABLE `baseall_explode_numbers` ( + `k3` int(11) NULL + ) ENGINE=OLAP + duplicate KEY(`k3`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k3`) BUCKETS 5 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "is_being_synced" = "false", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + sql "insert into baseall_explode_numbers values(1),(2),(3),(4),(5),(6),(7),(8),(9),(10),(11),(12),(13),(14),(15);" + qt_test4 """select k3,e from baseall_explode_numbers as U lateral view explode_numbers(5) tmp1 as e order by k3,e;""" + qt_test5 """select k3,e from baseall_explode_numbers as U lateral view explode_numbers(10) tmp1 as e order by k3,e;""" } diff --git a/regression-test/suites/query_p0/system/test_metadata_name_ids.groovy b/regression-test/suites/query_p0/system/test_metadata_name_ids.groovy new file mode 100644 index 000000000000000..ddce7c6a218a64c --- /dev/null +++ b/regression-test/suites/query_p0/system/test_metadata_name_ids.groovy @@ -0,0 +1,47 @@ +// 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_metadata_name_ids", "p0" ) { + + + def tableName = "internal.information_schema.metadata_name_ids" + qt_desc """ desc ${tableName} """ + + + sql """ create database if not exists demo; """ + sql """ use demo ; """ + + sql """ create table if not exists test_metadata_name_ids ( + a int , + b varchar(30) + ) + DUPLICATE KEY(`a`) + DISTRIBUTED BY HASH(`a`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + qt_select2 """ select CATALOG_NAME,DATABASE_NAME,TABLE_NAME from ${tableName} + where CATALOG_NAME="internal" and DATABASE_NAME ="demo" and TABLE_NAME="test_metadata_name_ids";""" + + sql """ drop table test_metadata_name_ids """ + + qt_select3 """ select CATALOG_NAME,DATABASE_NAME,TABLE_NAME from ${tableName} + where CATALOG_NAME="internal" and DATABASE_NAME ="demo" and TABLE_NAME="test_metadata_name_ids";""" + + +} diff --git a/regression-test/suites/query_p0/system/test_query_sys_tables.groovy b/regression-test/suites/query_p0/system/test_query_sys_tables.groovy index f74edcd52406ffd..c848f91ebca26f9 100644 --- a/regression-test/suites/query_p0/system/test_query_sys_tables.groovy +++ b/regression-test/suites/query_p0/system/test_query_sys_tables.groovy @@ -206,4 +206,5 @@ suite("test_query_sys_tables", "query,p0") { qt_sql "select * from key_column_usage" qt_sql "select * from triggers" qt_sql "select * from parameters" + qt_sql "select * from profiling" } \ No newline at end of file diff --git a/regression-test/suites/schema_change/test_alter_table_status.groovy b/regression-test/suites/schema_change/test_alter_table_status.groovy new file mode 100644 index 000000000000000..16e5cbcea4c5b7e --- /dev/null +++ b/regression-test/suites/schema_change/test_alter_table_status.groovy @@ -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. + +suite("test_alter_table_status") { + def tbName1 = "alter_table_status" + + try { + sql "DROP TABLE IF EXISTS ${tbName1}" + sql """ + CREATE TABLE IF NOT EXISTS ${tbName1} ( + k1 INT, + v1 INT, + v2 INT + ) + DUPLICATE KEY (k1) + DISTRIBUTED BY HASH(k1) BUCKETS 1 properties("replication_num" = "1", "light_schema_change" = "false", "disable_auto_compaction" = "true"); + """ + + // set table state to ROLLUP + sql """ADMIN SET TABLE ${tbName1} STATUS PROPERTIES ("state" = "rollup");""" + // try alter table comment + test { + sql """ ALTER TABLE ${tbName1} MODIFY COMMENT 'test'; """ + exception "Table[alter_table_status]'s state(ROLLUP) is not NORMAL. Do not allow doing ALTER ops" + } + + // set table state to NORMAL + sql """ADMIN SET TABLE ${tbName1} STATUS PROPERTIES ("state" = "normal");""" + // try alter table comment + sql """ ALTER TABLE ${tbName1} MODIFY COMMENT 'test'; """ + } finally { + // drop table + sql """ DROP TABLE ${tbName1} force""" + } +} diff --git a/regression-test/suites/schema_change_p0/test_agg_keys_schema_change.groovy b/regression-test/suites/schema_change_p0/test_agg_keys_schema_change.groovy index e6e0572d185e3fa..afc5b784fbe1379 100644 --- a/regression-test/suites/schema_change_p0/test_agg_keys_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_keys_schema_change.groovy @@ -63,7 +63,7 @@ suite ("test_agg_keys_schema_change") { `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); """ diff --git a/regression-test/suites/schema_change_p0/test_agg_mv_schema_change.groovy b/regression-test/suites/schema_change_p0/test_agg_mv_schema_change.groovy index c34522084acba2e..6d3f452967d430b 100644 --- a/regression-test/suites/schema_change_p0/test_agg_mv_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_mv_schema_change.groovy @@ -80,7 +80,7 @@ suite ("test_agg_mv_schema_change") { `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); """ diff --git a/regression-test/suites/schema_change_p0/test_agg_rollup_schema_change.groovy b/regression-test/suites/schema_change_p0/test_agg_rollup_schema_change.groovy index bee550e430a1517..ccd44b7b3db2738 100644 --- a/regression-test/suites/schema_change_p0/test_agg_rollup_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_rollup_schema_change.groovy @@ -80,7 +80,7 @@ suite ("test_agg_rollup_schema_change") { `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); """ diff --git a/regression-test/suites/schema_change_p0/test_agg_vals_schema_change.groovy b/regression-test/suites/schema_change_p0/test_agg_vals_schema_change.groovy index c5605a956586d7e..ebe882a543db507 100644 --- a/regression-test/suites/schema_change_p0/test_agg_vals_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_vals_schema_change.groovy @@ -61,7 +61,7 @@ suite ("test_agg_vals_schema_change") { `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); """ diff --git a/regression-test/suites/schema_change_p0/test_alter_table_column.groovy b/regression-test/suites/schema_change_p0/test_alter_table_column.groovy index 35807a5024bf443..21f147283a34c36 100644 --- a/regression-test/suites/schema_change_p0/test_alter_table_column.groovy +++ b/regression-test/suites/schema_change_p0/test_alter_table_column.groovy @@ -193,7 +193,7 @@ suite("test_alter_table_column") { `v1` int(11) SUM NULL COMMENT "" ) ENGINE=OLAP AGGREGATE KEY(`k1`, `k2`) - DISTRIBUTED BY HASH(`k1`) BUCKETS 1 + DISTRIBUTED BY HASH(`k1`) BUCKETS 5 PROPERTIES ( "storage_type" = "COLUMN", "replication_num" = "1" diff --git a/regression-test/suites/schema_change_p0/test_delete_schema_change.sql b/regression-test/suites/schema_change_p0/test_delete_schema_change.sql index 782561ada42d86c..d25cc58bce4d45f 100644 --- a/regression-test/suites/schema_change_p0/test_delete_schema_change.sql +++ b/regression-test/suites/schema_change_p0/test_delete_schema_change.sql @@ -13,6 +13,7 @@ CREATE TABLE IF NOT EXISTS schema_change_delete_regression_test ( `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); INSERT INTO schema_change_delete_regression_test VALUES diff --git a/regression-test/suites/schema_change_p0/test_dup_keys_schema_change.groovy b/regression-test/suites/schema_change_p0/test_dup_keys_schema_change.groovy index af3a4b45ae2fd36..dd2fa43f095a482 100644 --- a/regression-test/suites/schema_change_p0/test_dup_keys_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_dup_keys_schema_change.groovy @@ -62,7 +62,7 @@ suite ("test_dup_keys_schema_change") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); """ diff --git a/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy b/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy index ad510d698e0a2d2..168c450ebe141f3 100644 --- a/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy @@ -76,7 +76,7 @@ suite ("test_dup_mv_schema_change") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); """ diff --git a/regression-test/suites/schema_change_p0/test_dup_rollup_schema_change.groovy b/regression-test/suites/schema_change_p0/test_dup_rollup_schema_change.groovy index ebb78ec32b54425..1f1e0c5b61ac334 100644 --- a/regression-test/suites/schema_change_p0/test_dup_rollup_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_dup_rollup_schema_change.groovy @@ -80,7 +80,7 @@ suite ("test_dup_rollup_schema_change") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); """ diff --git a/regression-test/suites/schema_change_p0/test_dup_vals_schema_change.groovy b/regression-test/suites/schema_change_p0/test_dup_vals_schema_change.groovy index 716eb6bb3a71e73..6a3d9f61c9e5283 100644 --- a/regression-test/suites/schema_change_p0/test_dup_vals_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_dup_vals_schema_change.groovy @@ -57,7 +57,7 @@ suite ("test_dup_vals_schema_change") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); """ diff --git a/regression-test/suites/schema_change_p0/test_rename_column.groovy b/regression-test/suites/schema_change_p0/test_rename_column.groovy index 9ac575ca975b08c..558888954cfee75 100644 --- a/regression-test/suites/schema_change_p0/test_rename_column.groovy +++ b/regression-test/suites/schema_change_p0/test_rename_column.groovy @@ -40,6 +40,7 @@ suite ("test_rename_column") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + BUCKETS 8 PROPERTIES ( "replication_num" = "1" , "light_schema_change" = "false") """ qt_desc """ desc ${tableName} """ @@ -120,6 +121,7 @@ suite ("test_rename_column") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + BUCKETS 8 PROPERTIES ( "replication_num" = "1" , "light_schema_change" = "false") """ test { @@ -142,7 +144,7 @@ suite ("test_rename_column") { `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "true" ); """ qt_desc """ desc ${tableName} """ @@ -209,7 +211,7 @@ suite ("test_rename_column") { `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "true" ); """ diff --git a/regression-test/suites/schema_change_p0/test_uniq_keys_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_keys_schema_change.groovy index 3c6778f1a64be19..d3eaec8cfa3cd94 100644 --- a/regression-test/suites/schema_change_p0/test_uniq_keys_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_uniq_keys_schema_change.groovy @@ -57,7 +57,7 @@ suite ("test_uniq_keys_schema_change") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false"); """ diff --git a/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy index 7408e6c1d1c8d18..71c324a7294a618 100644 --- a/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy @@ -75,7 +75,7 @@ suite ("test_uniq_mv_schema_change") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false", 'enable_unique_key_merge_on_write' = 'false'); """ diff --git a/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy index 821a2294879c2ab..52d4799807edd0d 100644 --- a/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy @@ -79,7 +79,7 @@ suite ("test_uniq_rollup_schema_change") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false", 'enable_unique_key_merge_on_write' = 'false'); """ diff --git a/regression-test/suites/schema_change_p0/test_uniq_seq_col_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_seq_col_schema_change.groovy index d1adc9e0eb8551e..be3bc1025688ee1 100644 --- a/regression-test/suites/schema_change_p0/test_uniq_seq_col_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_uniq_seq_col_schema_change.groovy @@ -29,7 +29,7 @@ suite("test_uniq_seq_col_schema_change", "schema_change") { value3 INT ) UNIQUE KEY (k1) - DISTRIBUTED BY HASH(k1) BUCKETS 1 + DISTRIBUTED BY HASH(k1) BUCKETS 8 properties("replication_num" = "1", "light_schema_change" = "false", "function_column.sequence_type" = 'INT'); diff --git a/regression-test/suites/schema_change_p0/test_uniq_vals_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_vals_schema_change.groovy index a9ee1ddab423524..43c08dc7300f86b 100644 --- a/regression-test/suites/schema_change_p0/test_uniq_vals_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_uniq_vals_schema_change.groovy @@ -59,7 +59,7 @@ suite ("test_uniq_vals_schema_change") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) - BUCKETS 1 + BUCKETS 8 PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false" ); """ diff --git a/regression-test/suites/schema_change_p0/test_update_schema_change.groovy b/regression-test/suites/schema_change_p0/test_update_schema_change.groovy index 74c787f976814f9..aa375b274c39981 100644 --- a/regression-test/suites/schema_change_p0/test_update_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_update_schema_change.groovy @@ -38,6 +38,7 @@ suite ("test_update_schema_change") { `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + BUCKETS 8 PROPERTIES ( "replication_num" = "1" , "light_schema_change" = "false"); """ diff --git a/regression-test/suites/ssb_sf0.1_p1/load.groovy b/regression-test/suites/ssb_sf0.1_p1/load.groovy index 5151bff68e1077d..3b3955dc3ac7e63 100644 --- a/regression-test/suites/ssb_sf0.1_p1/load.groovy +++ b/regression-test/suites/ssb_sf0.1_p1/load.groovy @@ -119,4 +119,5 @@ suite("load") { rowCount = sql "select count(*) from ${table}" assertEquals(table_rows, rowCount[0][0]) } + sql """ sync """ } diff --git a/regression-test/suites/ssb_unique_sql_zstd_p0/load.groovy b/regression-test/suites/ssb_unique_sql_zstd_p0/load.groovy index 1c51fdd56e86681..5e0a460569cb346 100644 --- a/regression-test/suites/ssb_unique_sql_zstd_p0/load.groovy +++ b/regression-test/suites/ssb_unique_sql_zstd_p0/load.groovy @@ -77,4 +77,6 @@ suite("load") { } i++ } + + sql """ sync """ } diff --git a/regression-test/suites/statistics/analyze_stats.groovy b/regression-test/suites/statistics/analyze_stats.groovy index 40f772025692d4d..50420613b5de289 100644 --- a/regression-test/suites/statistics/analyze_stats.groovy +++ b/regression-test/suites/statistics/analyze_stats.groovy @@ -143,99 +143,99 @@ suite("test_analyze") { ANALYZE DATABASE ${db} WITH SYNC WITH SAMPLE PERCENT 10 """ -// a_result_2 = sql """ -// ANALYZE DATABASE ${db} WITH SYNC WITH SAMPLE PERCENT 5 -// """ -// -// a_result_3 = sql """ -// ANALYZE DATABASE ${db} WITH SAMPLE PERCENT 5 -// """ -// -// show_result = sql """ -// SHOW ANALYZE -// """ -// -// def contains_expected_table = { r -> -// for (int i = 0; i < r.size; i++) { -// if (r[i][3] == "${tbl}") { -// return true -// } -// } -// return false -// } -// -// def stats_job_removed = { r, id -> -// for (int i = 0; i < r.size; i++) { -// if (r[i][0] == id) { -// return false -// } -// } -// return true -// } -// -// assert contains_expected_table(show_result) -// -// sql """ -// DROP ANALYZE JOB ${a_result_3[0][4]} -// """ -// -// show_result = sql """ -// SHOW ANALYZE -// """ -// -// assert stats_job_removed(show_result, a_result_3[0][4]) -// -// sql """ -// ANALYZE DATABASE ${db} WITH SAMPLE ROWS 5 WITH PERIOD 100000 -// """ -// -// sql """ -// DROP TABLE IF EXISTS analyze_partitioned_tbl_test -// """ - -// sql """ -// CREATE TABLE analyze_partitioned_tbl_test (col1 int, col2 int, col3 int) -// PARTITION BY RANGE(`col2`) ( -// PARTITION `p1` VALUES LESS THAN ('5'), -// PARTITION `p2` VALUES LESS THAN ('10'), -// PARTITION `P3` VALUES LESS THAN ('15'), -// PARTITION `P4` VALUES LESS THAN ('20'), -// PARTITION `P5` VALUES LESS THAN ('25'), -// PARTITION `P6` VALUES LESS THAN ('30')) -// DISTRIBUTED BY HASH(col3) -// BUCKETS 3 -// PROPERTIES( -// "replication_num"="1" -// ) -// """ -// -// sql """insert into analyze_partitioned_tbl_test values(1,3,1) """ -// sql """insert into analyze_partitioned_tbl_test values(6,6,6) """ -// sql """insert into analyze_partitioned_tbl_test values(11,6,6) """ -// sql """insert into analyze_partitioned_tbl_test values(16,6,6) """ -// sql """insert into analyze_partitioned_tbl_test values(21,6,6) """ -// sql """insert into analyze_partitioned_tbl_test values(26,6,6) """ -// -// sql """ -// ANALYZE TABLE analyze_partitioned_tbl_test WITH SYNC -// """ -// -// part_tbl_analyze_result = sql """ -// SHOW COLUMN CACHED STATS analyze_partitioned_tbl_test(col1) -// """ -// -// def expected_result = { r -> -// for (int i = 0; i < r.size; i++) { -// if ((int) Double.parseDouble(r[i][1]) == 6) { -// return true -// } else { -// return false -// } -// } -// return false -// } -// -// assert expected_result(part_tbl_analyze_result) + a_result_2 = sql """ + ANALYZE DATABASE ${db} WITH SYNC WITH SAMPLE PERCENT 5 + """ + + a_result_3 = sql """ + ANALYZE DATABASE ${db} WITH SAMPLE PERCENT 5 + """ + + show_result = sql """ + SHOW ANALYZE + """ + + def contains_expected_table = { r -> + for (int i = 0; i < r.size; i++) { + if (r[i][3] == "${tbl}") { + return true + } + } + return false + } + + def stats_job_removed = { r, id -> + for (int i = 0; i < r.size; i++) { + if (r[i][0] == id) { + return false + } + } + return true + } + + assert contains_expected_table(show_result) + + sql """ + DROP ANALYZE JOB ${a_result_3[0][4]} + """ + + show_result = sql """ + SHOW ANALYZE + """ + + assert stats_job_removed(show_result, a_result_3[0][4]) + + sql """ + ANALYZE DATABASE ${db} WITH SAMPLE ROWS 5 WITH PERIOD 100000 + """ + + sql """ + DROP TABLE IF EXISTS analyze_partitioned_tbl_test + """ + + sql """ + CREATE TABLE analyze_partitioned_tbl_test (col1 int, col2 int, col3 int) + PARTITION BY RANGE(`col2`) ( + PARTITION `p1` VALUES LESS THAN ('5'), + PARTITION `p2` VALUES LESS THAN ('10'), + PARTITION `P3` VALUES LESS THAN ('15'), + PARTITION `P4` VALUES LESS THAN ('20'), + PARTITION `P5` VALUES LESS THAN ('25'), + PARTITION `P6` VALUES LESS THAN ('30')) + DISTRIBUTED BY HASH(col3) + BUCKETS 3 + PROPERTIES( + "replication_num"="1" + ) + """ + + sql """insert into analyze_partitioned_tbl_test values(1,3,1) """ + sql """insert into analyze_partitioned_tbl_test values(6,6,6) """ + sql """insert into analyze_partitioned_tbl_test values(11,6,6) """ + sql """insert into analyze_partitioned_tbl_test values(16,6,6) """ + sql """insert into analyze_partitioned_tbl_test values(21,6,6) """ + sql """insert into analyze_partitioned_tbl_test values(26,6,6) """ + + sql """ + ANALYZE TABLE analyze_partitioned_tbl_test WITH SYNC + """ + + part_tbl_analyze_result = sql """ + SHOW COLUMN CACHED STATS analyze_partitioned_tbl_test(col1) + """ + + def expected_result = { r -> + for (int i = 0; i < r.size; i++) { + if ((int) Double.parseDouble(r[i][1]) == 6) { + return true + } else { + return false + } + } + return false + } + + assert expected_result(part_tbl_analyze_result) sql """ DROP TABLE IF EXISTS test_600_partition_table_analyze; @@ -877,4 +877,31 @@ PARTITION `p599` VALUES IN (599) assert expected_id_col_stats(id_col_stats, 600, 1) assert expected_id_col_stats(id_col_stats, 599, 7) assert expected_id_col_stats(id_col_stats, 0, 6) + + sql """DROP TABLE IF EXISTS increment_analyze_test""" + sql """ + CREATE TABLE increment_analyze_test ( + id BIGINT + ) DUPLICATE KEY(`id`) + PARTITION BY RANGE(`id`) + ( + PARTITION `p1` VALUES LESS THAN ('5') + ) + + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( + "replication_num"="1" + ); + """ + + sql """INSERT INTO increment_analyze_test VALUES(1),(2),(3)""" + sql """ANALYZE TABLE increment_analyze_test WITH SYNC""" + sql """ALTER TABLE increment_analyze_test ADD PARTITION p2 VALUES LESS THAN('10')""" + + sql """INSERT INTO increment_analyze_test VALUES(6),(7),(8)""" + sql """ANALYZE TABLE increment_analyze_test WITH SYNC WITH INCREMENTAL""" + def inc_res = sql """ + SHOW COLUMN CACHED STATS increment_analyze_test(id) + """ + expected_id_col_stats(inc_res, 6, 1) } \ No newline at end of file diff --git a/regression-test/suites/tpcds_sf1_p1/load.groovy b/regression-test/suites/tpcds_sf1_p1/load.groovy index d44fd13e1125b9f..5e1422b58d81044 100644 --- a/regression-test/suites/tpcds_sf1_p1/load.groovy +++ b/regression-test/suites/tpcds_sf1_p1/load.groovy @@ -100,4 +100,5 @@ suite("load") { } sql """ ANALYZE TABLE $tableName WITH SYNC """ } + sql """ sync """ } diff --git a/regression-test/suites/tpcds_sf1_p2/load.groovy b/regression-test/suites/tpcds_sf1_p2/load.groovy index e8275d0fb4dfa15..5df694c6b49c414 100644 --- a/regression-test/suites/tpcds_sf1_p2/load.groovy +++ b/regression-test/suites/tpcds_sf1_p2/load.groovy @@ -69,4 +69,5 @@ suite("load") { } sql """ ANALYZE TABLE $tableName WITH SYNC """ } + sql """ sync """ } diff --git a/regression-test/suites/tpcds_sf1_unique_p1/load.groovy b/regression-test/suites/tpcds_sf1_unique_p1/load.groovy index bfe7b9fd3120caf..9f7f7d198ced909 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/load.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/load.groovy @@ -152,4 +152,6 @@ suite("load") { sql "insert into tt select * from t" new_count = sql "select count(*) from tt" assertEquals(origin_count, new_count) + + sql """ sync """ } diff --git a/regression-test/suites/tpch_sf0.1_p1/load.groovy b/regression-test/suites/tpch_sf0.1_p1/load.groovy index 9b8d54a4da7d8ea..3e538056d727f2c 100644 --- a/regression-test/suites/tpch_sf0.1_p1/load.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/load.groovy @@ -78,4 +78,5 @@ suite("load") { def table = "revenue1" sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text sql new File("""${context.file.parent}/ddl/${table}.sql""").text + sql """ sync """ } diff --git a/regression-test/suites/tpch_sf0.1_unique_p1/load.groovy b/regression-test/suites/tpch_sf0.1_unique_p1/load.groovy index 6bc41d49f958323..9e9d511e26df2f6 100644 --- a/regression-test/suites/tpch_sf0.1_unique_p1/load.groovy +++ b/regression-test/suites/tpch_sf0.1_unique_p1/load.groovy @@ -80,4 +80,6 @@ suite("load") { def table = "revenue1" sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text sql new File("""${context.file.parent}/ddl/${table}.sql""").text + + sql """ sync """ } diff --git a/regression-test/suites/tpch_sf1_p2/load.groovy b/regression-test/suites/tpch_sf1_p2/load.groovy index db5e5dc77565708..c7136d265dddb30 100644 --- a/regression-test/suites/tpch_sf1_p2/load.groovy +++ b/regression-test/suites/tpch_sf1_p2/load.groovy @@ -118,4 +118,6 @@ suite("load") { // We need wait to make sure BE could pass the stats info to FE so that // avoid unnessary inconsistent generated plan which would cause the regression test fail sleep(60000) + + sql """ sync """ } diff --git a/regression-test/suites/tpch_sf1_unique_p2/load.groovy b/regression-test/suites/tpch_sf1_unique_p2/load.groovy index 26e72671259ed5e..cccc988ae94d337 100644 --- a/regression-test/suites/tpch_sf1_unique_p2/load.groovy +++ b/regression-test/suites/tpch_sf1_unique_p2/load.groovy @@ -103,4 +103,5 @@ suite("load") { def table = "revenue1" sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text sql new File("""${context.file.parent}/ddl/${table}.sql""").text + sql """ sync """ } diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/load.groovy b/regression-test/suites/tpch_unique_sql_zstd_p0/load.groovy index 885330c616d4b0c..444c7d413ef4863 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/load.groovy +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/load.groovy @@ -79,4 +79,6 @@ suite("load") { def table = "revenue1" sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text sql new File("""${context.file.parent}/ddl/${table}.sql""").text + + sql """ sync """ } diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update.groovy index b8052cf5255c35a..6af0e8694828067 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update.groovy @@ -166,7 +166,7 @@ suite("test_primary_key_partial_update", "p0") { assertTrue(exception == null) def json = parseJson(result) assertEquals("Fail", json.Status) - assertEquals("[INTERNAL_ERROR]too many filtered rows", json.Message) + assertTrue(json.Message.contains("[INTERNAL_ERROR]too many filtered rows")) assertEquals(3, json.NumberTotalRows) assertEquals(1, json.NumberLoadedRows) assertEquals(2, json.NumberFilteredRows) diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_strict_mode.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_strict_mode.groovy index 9262d7116f7b720..256cf018d54d1c0 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_strict_mode.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_strict_mode.groovy @@ -108,7 +108,7 @@ suite("test_partial_update_strict_mode", "p0") { assertTrue(exception == null) def json = parseJson(result) assertEquals("Fail", json.Status) - assertEquals("[INTERNAL_ERROR]too many filtered rows", json.Message) + assertTrue(json.Message.contains("[INTERNAL_ERROR]too many filtered rows")) assertEquals(3, json.NumberTotalRows) assertEquals(1, json.NumberLoadedRows) assertEquals(2, json.NumberFilteredRows) @@ -157,7 +157,7 @@ suite("test_partial_update_strict_mode", "p0") { assertTrue(exception == null) def json = parseJson(result) assertEquals("Fail", json.Status) - assertEquals("[INTERNAL_ERROR]too many filtered rows", json.Message) + assertTrue(json.Message.contains("[INTERNAL_ERROR]too many filtered rows")) assertEquals(3, json.NumberTotalRows) assertEquals(1, json.NumberLoadedRows) assertEquals(2, json.NumberFilteredRows) diff --git a/regression-test/suites/version_p0/test_set_partition_version.groovy b/regression-test/suites/version_p0/test_set_partition_version.groovy new file mode 100644 index 000000000000000..b461d01d800eaec --- /dev/null +++ b/regression-test/suites/version_p0/test_set_partition_version.groovy @@ -0,0 +1,71 @@ +// 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_set_partition_version") { + def tableName1 = "test_set_partition_version" + sql """ DROP TABLE IF EXISTS ${tableName1} """ + sql """ + CREATE TABLE ${tableName1} ( + `id` int NOT NULL, + `version` int NOT NULL COMMENT '插入次数' + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES + ( + "replication_allocation" = "tag.location.default: 1", + "disable_auto_compaction" = "true" + ); + """ + + def res = sql """ show partitions from ${tableName1}; """ + def partitionId = res[0][0].toString() + + // load 1 time, partition visible version should be 2 + sql """ insert into ${tableName1} values (1, 2); """ + res = sql """ show partitions from ${tableName1}; """ + assertEquals(res[0][2].toString(), "2") + + // load 2 time, partition visible version should be 3 + sql """ insert into ${tableName1} values (2, 3); """ + res = sql """ show partitions from ${tableName1}; """ + assertEquals(res[0][2].toString(), "3") + + // set partition visible version to 2 + sql """ ADMIN SET TABLE ${tableName1} PARTITION VERSION PROPERTIES ("partition_id" = "${partitionId}", "visible_version" = "2"); """ + res = sql """ show partitions from ${tableName1}; """ + assertEquals(res[0][2].toString(), "2") + + // check if table can query, and return row size should be 1 + res = sql """ select * from ${tableName1}; """ + assertEquals(res.size(), 1) + + // set partition visible version to 3 + sql """ ADMIN SET TABLE ${tableName1} PARTITION VERSION PROPERTIES ("partition_id" = "${partitionId}", "visible_version" = "3"); """ + res = sql """ show partitions from ${tableName1}; """ + assertEquals(res[0][2].toString(), "3") + + // check if table can query, and return row size should be 2 + res = sql """ select * from ${tableName1}; """ + assertEquals(res.size(), 2) + + // load 3 time, partition visible version should be 4 + sql """ insert into ${tableName1} values (3, 4); """ + res = sql """ show partitions from ${tableName1}; """ + assertEquals(res[0][2].toString(), "4") +} diff --git a/regression-test/suites/view_p0/view_p0.groovy b/regression-test/suites/view_p0/view_p0.groovy index a6f20d41024ac86..bb6a5ab5a631e03 100644 --- a/regression-test/suites/view_p0/view_p0.groovy +++ b/regression-test/suites/view_p0/view_p0.groovy @@ -112,5 +112,15 @@ suite("view_p0") { ) c; """ qt_sql "select * from test_element_at_view;" + + sql "drop view if exists test_element_at_view" + + sql "drop view if exists test_time_diff" + + sql "create view test_time_diff as select minutes_diff('2023-01-16 10:05:04', '2023-01-15 18:05:04')" + + qt_sql "select * from test_time_diff" + + sql "drop view if exists test_time_diff" } diff --git a/run-be-ut.sh b/run-be-ut.sh index 02fe5f368ed3f46..a0ab31f83df760c 100755 --- a/run-be-ut.sh +++ b/run-be-ut.sh @@ -201,7 +201,7 @@ cd "${CMAKE_BUILD_DIR}" -DUSE_LIBCPP="${USE_LIBCPP}" \ -DBUILD_META_TOOL=OFF \ -DBUILD_BENCHMARK_TOOL="${BUILD_BENCHMARK_TOOL}" \ - -DWITH_MYSQL=OFF \ + -DWITH_MYSQL=ON \ -DUSE_DWARF="${USE_DWARF}" \ -DUSE_UNWIND="${USE_UNWIND}" \ -DUSE_MEM_TRACKER="${USE_MEM_TRACKER}" \ @@ -381,11 +381,20 @@ export JAVA_OPTS="-Xmx1024m -DlogPath=${DORIS_HOME}/log/jni.log -Xloggc:${DORIS_ # find all executable test files test="${DORIS_TEST_BINARY_DIR}/doris_be_test" profraw=${DORIS_TEST_BINARY_DIR}/doris_be_test.profraw +profdata=${DORIS_TEST_BINARY_DIR}/doris_be_test.profdata file_name="${test##*/}" if [[ -f "${test}" ]]; then if [[ "_${DENABLE_CLANG_COVERAGE}" == "_ON" ]]; then LLVM_PROFILE_FILE="${profraw}" "${test}" --gtest_output="xml:${GTEST_OUTPUT_DIR}/${file_name}.xml" --gtest_print_time=true "${FILTER}" + if [[ -d "${DORIS_TEST_BINARY_DIR}"/report ]]; then + rm -rf "${DORIS_TEST_BINARY_DIR}"/report + fi + "${LLVM_PROFDATA} merge -o ${profdata} ${profraw}" + "${LLVM_COV} show -output-dir=${DORIS_TEST_BINARY_DIR}/report -format=html \ + -ignore-filename-regex='(.*gensrc/.*)|(.*_test\.cpp$)|(.*be/test.*)|(.*apache-orc/.*)|(.*clucene/.*)' \ + -instr-profile=${profdata} \ + -object=${test}" else "${test}" --gtest_output="xml:${GTEST_OUTPUT_DIR}/${file_name}.xml" --gtest_print_time=true "${FILTER}" fi